You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/29 13:29:38 UTC

[GitHub] [flink] lincoln-lil opened a new pull request, #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

lincoln-lil opened a new pull request, #20393:
URL: https://github.com/apache/flink/pull/20393

   ## What is the purpose of the change
   this is two main parts of the parent issue FLINK-27849
   1. FLINK-28570: Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error
    which introduce a new optimizer option: "table.optimizer.non-deterministic-update.handling" with default value  `IGNORE` to keep the compatibility. When user cares about the effects of non-deterministic updates, the option can be set to `TRY_RESOLVE`, then the inner `StreamNonDeterministicPlanResolver` will validate if there's any non-deterministic updates which may cause wrong result or error, and also try to eliminate the non determinism generated by lookup join node by adding materialization to a new physical lookup joins operator, will raise an error if there still exists other non-deterministic beside lookup join
   
   2. FLINK-28568:  Implements a new lookup join operator (sync mode only) with state to eliminate the non determinism
    which is a followup issue of FLINK-28570 to finish the materialization support in runtime, note that only sync lookup function is supported for now due to some runtime limitation (async lookup join based on the AsyncWaitOperator which has operatror state, and the materialization requirement here needs a keyed state, so unless we invent a new KeyedAsyncWaitOperator here, the work can not be done for now)
   
   ## Brief change log
   * add 'table.optimizer.non-deterministic-update.handling' to OptimizerConfigOptions
   * implement a StreamNonDeterministicPlanResolver to visit all stream physical node and try to rewrite lookup join node with materialization enabled
   * add new tests and update existing tests to best effortly covering the new logic 
   
   ## Verifying this change
   NonDeterministicDagTest KeyedLookupJoinHarnessTest and existing tests
   
   
   ## Does this pull request potentially affect one of the following parts:
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
     - The serializers: (no )
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (docs)


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r936658329


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {

Review Comment:
   sigh... there're different logic for groupAgg, overAgg, windowAgg, regular join and source node



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1206207068

   @flinkbot run azure


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] godfreyhe commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r935451881


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,19 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {
+      case streamRel: StreamPhysicalRel =>
+        val fmq = FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+        val upsertKeys = fmq.getUpsertKeys(streamRel)
+        if (null != upsertKeys && !upsertKeys.isEmpty) {
+          printValues.add(
+            Pair.of("upsertKeys", upsertKeys.map(bitset => bitset.toString).mkString(", ")))

Review Comment:
   fields name is better than index



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,19 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {

Review Comment:
   please add some tests in `FlinkRelOptUtilTest`



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala:
##########
@@ -80,7 +81,8 @@ object FlinkRelOptUtil {
       withIdPrefix,
       withChangelogTraits,
       withRowType,
-      withTreeStyle = true)
+      withTreeStyle = true,
+      withUpsertKey)

Review Comment:
   nit: move `withUpsertKey` before `withTreeStyle `



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala:
##########
@@ -62,14 +62,19 @@ class StreamPhysicalIncrementalGroupAggregate(
     inputRel: RelNode,
     val partialAggGrouping: Array[Int],
     val partialAggCalls: Array[AggregateCall],
-    val finalAggGrouping: Array[Int],

Review Comment:
   I think `finalAggGrouping` is clearer than `grouping`, because there are two kind of grouping



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {

Review Comment:
   nit: temporalPkIdxs.nonEmpty



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {
+        val program = calcOnTemporalTable.get
+        val outputMapping = program.getProjectList.asScala.zipWithIndex
+          .map { case (ref, index) => (index, program.expandLocalRef(ref)) }
+          .map {
+            case (outIndex, ref) =>
+              ref match {
+                case inputRef: RexInputRef => (inputRef.getIndex, outIndex)
+                case _ => (-1, -1)
+              }
+          }
+          .toMap
+        val outputPk = temporalPkIdxs.forall(outputMapping.contains)
+        if (outputPk) {
+          // remapping pk index
+          temporalPkIdxs.map(outputMapping.get(_).get)
+        } else {
+          Array[Int]()
+        }
+      } else {
+        temporalPkIdxs
+      }
+    } else {
+      // temporal table has no pk, no uk produces
+      Array[Int]()
+    }
+
+    outputPkIdx
+  }
+
+  private def getPrimaryKeyIndexesOfTemporalTable: Array[Int] = {
+    // get primary key columns of lookup table if exists
+    val pkColumns = getPrimaryKeyColumnsOfTemporalTable
+    if (pkColumns.isDefined) {
+      val newSchema = temporalTable.getRowType.getFieldNames
+      pkColumns.get.toArray().map(col => newSchema.indexOf(col))
+    } else {
+      Array[Int]()
+    }
+  }
+
+  private def getPrimaryKeyColumnsOfTemporalTable: Option[util.List[String]] = {
+    val schema = temporalTable match {
+      case t: TableSourceTable =>
+        TableSchema.fromResolvedSchema(t.contextResolvedTable.getResolvedSchema)

Review Comment:
   TableSchema is a deprecated class, it's better we could avoid use it



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {

Review Comment:
   two many if-else here, we can use `match` to avoid it



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {
+        val program = calcOnTemporalTable.get
+        val outputMapping = program.getProjectList.asScala.zipWithIndex
+          .map { case (ref, index) => (index, program.expandLocalRef(ref)) }
+          .map {
+            case (outIndex, ref) =>
+              ref match {
+                case inputRef: RexInputRef => (inputRef.getIndex, outIndex)
+                case _ => (-1, -1)
+              }
+          }
+          .toMap
+        val outputPk = temporalPkIdxs.forall(outputMapping.contains)
+        if (outputPk) {
+          // remapping pk index
+          temporalPkIdxs.map(outputMapping.get(_).get)

Review Comment:
   outputMapping.get(_).get => outputMapping(_)



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {
+      val rightUniqueKeys = getUniqueKeysOfTemporalTable(join)
+
+      getJoinUniqueKeys(
+        join.joinType,
+        left.getRowType,
+        leftUniqueKeys,
+        rightUniqueKeys,
+        mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
+        rightUniqueKeys != null)
+    } else {
       null
-    )
+    }
+  }
+
+  private[flink] def getUniqueKeysOfTemporalTable(

Review Comment:
   could this method and CommonPhysicalLookupJoin#lookupKeyContainsPrimaryKey be combined into one ?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val rightUpsertKeys =
+      if (
+        (join.remainingCondition.isDefined && !FlinkRexUtil.isDeterministicInStreaming(
+          join.remainingCondition.get))
+        || (join.calcOnTemporalTable.isDefined && !FlinkRexUtil.isDeterministicInStreaming(

Review Comment:
   `isDeterministicInStreaming` is introduce in the next pr, this commit is invalid.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val rightUpsertKeys =
+      if (
+        (join.remainingCondition.isDefined && !FlinkRexUtil.isDeterministicInStreaming(

Review Comment:
   The if condition is too long, assign the result to a variable, and the if-else will become clearer



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {

Review Comment:
   getOutputPrimaryKeyIndexesOfTemporalTable



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val rightUpsertKeys =
+      if (
+        (join.remainingCondition.isDefined && !FlinkRexUtil.isDeterministicInStreaming(

Review Comment:
   join.remainingCondition.exists(c => !FlinkRexUtil.isDeterministicInStreaming(c))



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {

Review Comment:
   It's better we can add tests for cover the changes in the commit



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java:
##########
@@ -142,4 +142,33 @@ public class OptimizerConfigOptions {
                     .withDescription(
                             "When it is true, the optimizer will merge the operators with pipelined shuffling "
                                     + "into a multiple input operator to reduce shuffling and improve performance. Default value is true.");
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<NonDeterministicUpdateHandling>
+            TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_HANDLING =
+                    key("table.optimizer.non-deterministic-update.handling")

Review Comment:
   how about `table.optimizer.non-deterministic-update-strategy` ?
   
   if this config option only used for lookup join, the config name should contain lookup-join keyword



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {
+    val visitor = new RexVisitorImpl[Void](true) {
+      override def visitCall(call: RexCall): Void = {
+        // dynamic function call is also non-deterministic to streaming
+        if (!call.getOperator.isDeterministic || call.getOperator.isDynamicFunction)

Review Comment:
   add {}



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalJoin.scala:
##########
@@ -80,4 +81,14 @@ abstract class CommonPhysicalJoin(
       .item("select", getRowType.getFieldNames.mkString(", "))
   }
 
+  def getUniqueKeys(input: RelNode, keys: Array[Int]): List[Array[Int]] = {

Review Comment:
   getUpsertKeys ?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {

Review Comment:
   Why we emphasize `streaming` ?
   
   we should add test cases for this util method



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {
+    val visitor = new RexVisitorImpl[Void](true) {
+      override def visitCall(call: RexCall): Void = {
+        // dynamic function call is also non-deterministic to streaming
+        if (!call.getOperator.isDeterministic || call.getOperator.isDynamicFunction)
+          throw Util.FoundOne.NULL
+        super.visitCall(call)
+      }
+    }
+    e.accept(visitor)
+    true
+  } catch {
+    case ex: Util.FoundOne =>
+      Util.swallow(ex, null)
+      false
+  }
+
+  /**
+   * Returns whether a given [[RexProgram]] is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   * @return
+   *   true if any expression of the program is not deterministic in streaming
+   */
+  def isDeterministicInStreaming(rexProgram: RexProgram): Boolean = try {
+    if (null != rexProgram.getCondition) {
+      val rexCondi = rexProgram.expandLocalRef(rexProgram.getCondition)
+      if (!isDeterministicInStreaming(rexCondi)) {
+        return false
+      }
+    }
+    val projects = rexProgram.getProjectList.map(rexProgram.expandLocalRef)
+    projects.forall {
+      expr =>
+        expr match {
+          case rexNode: RexNode => isDeterministicInStreaming(rexNode)
+          case _ => true // ignore
+        }

Review Comment:
   these line can be simplified as `projects.forall(isDeterministicInStreaming)`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] godfreyhe commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r936612711


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {

Review Comment:
   I think it also can be used in batch, because they all generate non-deterministic result



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r935658039


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala:
##########
@@ -62,14 +62,19 @@ class StreamPhysicalIncrementalGroupAggregate(
     inputRel: RelNode,
     val partialAggGrouping: Array[Int],
     val partialAggCalls: Array[AggregateCall],
-    val finalAggGrouping: Array[Int],

Review Comment:
   Do you mean the attribute name in `StreamPhysicalGroupAggregateBase`?
   I was thinking the partial-final is the internal concept for two-phase aggregate, so tend to omit the 'final' prefix in the base class which I think can both represents
   grouping in one-phase agg and distinguishes from partialGroupings in two-phase agg.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,19 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {
+      case streamRel: StreamPhysicalRel =>
+        val fmq = FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+        val upsertKeys = fmq.getUpsertKeys(streamRel)
+        if (null != upsertKeys && !upsertKeys.isEmpty) {
+          printValues.add(
+            Pair.of("upsertKeys", upsertKeys.map(bitset => bitset.toString).mkString(", ")))

Review Comment:
   The previous idea using index for simplicity since there already has field names, but upsertKey usually not too long, I'll update it



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {

Review Comment:
   I want to highlight it to avoid some one using it in batch processing due to the different semantics on dynamic function.
   I'll add separate tests for it.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val rightUpsertKeys =
+      if (
+        (join.remainingCondition.isDefined && !FlinkRexUtil.isDeterministicInStreaming(
+          join.remainingCondition.get))
+        || (join.calcOnTemporalTable.isDefined && !FlinkRexUtil.isDeterministicInStreaming(

Review Comment:
   good catch! I'll re-split the commits for next updates



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {

Review Comment:
   this output indexes is the final one which may has a project inside, not directly on a temporal table, so I didn't add the temporalTable into the name.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala:
##########
@@ -80,7 +81,8 @@ object FlinkRelOptUtil {
       withIdPrefix,
       withChangelogTraits,
       withRowType,
-      withTreeStyle = true)
+      withTreeStyle = true,
+      withUpsertKey)

Review Comment:
   I had the same feeling at first, but all attributes in `RelTreeWriterImpl` has default values, so I appended the new `withUpsertKey` to the last position of `RelTreeWriterImpl`.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java:
##########
@@ -142,4 +142,33 @@ public class OptimizerConfigOptions {
                     .withDescription(
                             "When it is true, the optimizer will merge the operators with pipelined shuffling "
                                     + "into a multiple input operator to reduce shuffling and improve performance. Default value is true.");
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<NonDeterministicUpdateHandling>
+            TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_HANDLING =
+                    key("table.optimizer.non-deterministic-update.handling")

Review Comment:
   This option aimed for resolving all NDU (non deterministic update) problems in the long run, lookup join has the first priority in the goal. 
   For naming, the main consideration is similar to error-handling, NDU is also one kind of errors, so I prefer handling than strategy, WDYT?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,19 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {

Review Comment:
   Ok



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala:
##########
@@ -62,14 +62,19 @@ class StreamPhysicalIncrementalGroupAggregate(
     inputRel: RelNode,
     val partialAggGrouping: Array[Int],
     val partialAggCalls: Array[AggregateCall],
-    val finalAggGrouping: Array[Int],

Review Comment:
   Do you mean the attribute name in `StreamPhysicalGroupAggregateBase`?
   I was thinking the partial-final is the internal concept for two-phase aggregate, so tend to omit the 'final' prefix in the base class which I think can both represents
   grouping in one-phase agg and distinguishes from partialGroupings in two-phase agg.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalJoin.scala:
##########
@@ -80,4 +81,14 @@ abstract class CommonPhysicalJoin(
       .item("select", getRowType.getFieldNames.mkString(", "))
   }
 
+  def getUniqueKeys(input: RelNode, keys: Array[Int]): List[Array[Int]] = {

Review Comment:
   this method was moved from `StreamPhysicalJoin`, so I didn't change the original name (there're several related methods named `xxUniqueKeys`, maybe better refact this in another pr)



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {
+        val program = calcOnTemporalTable.get
+        val outputMapping = program.getProjectList.asScala.zipWithIndex
+          .map { case (ref, index) => (index, program.expandLocalRef(ref)) }
+          .map {
+            case (outIndex, ref) =>
+              ref match {
+                case inputRef: RexInputRef => (inputRef.getIndex, outIndex)
+                case _ => (-1, -1)
+              }
+          }
+          .toMap
+        val outputPk = temporalPkIdxs.forall(outputMapping.contains)
+        if (outputPk) {
+          // remapping pk index
+          temporalPkIdxs.map(outputMapping.get(_).get)
+        } else {
+          Array[Int]()
+        }
+      } else {
+        temporalPkIdxs
+      }
+    } else {
+      // temporal table has no pk, no uk produces
+      Array[Int]()
+    }
+
+    outputPkIdx
+  }
+
+  private def getPrimaryKeyIndexesOfTemporalTable: Array[Int] = {
+    // get primary key columns of lookup table if exists
+    val pkColumns = getPrimaryKeyColumnsOfTemporalTable
+    if (pkColumns.isDefined) {
+      val newSchema = temporalTable.getRowType.getFieldNames
+      pkColumns.get.toArray().map(col => newSchema.indexOf(col))
+    } else {
+      Array[Int]()
+    }
+  }
+
+  private def getPrimaryKeyColumnsOfTemporalTable: Option[util.List[String]] = {
+    val schema = temporalTable match {
+      case t: TableSourceTable =>
+        TableSchema.fromResolvedSchema(t.contextResolvedTable.getResolvedSchema)

Review Comment:
   This is mainly due to the desire to obtain pk in a uniform way, but not found proper method on the `LegacyTableSourceTable`. We can switch to use ResolvedSchema after `LegacyTableSourceTable` was removed



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val rightUpsertKeys =
+      if (
+        (join.remainingCondition.isDefined && !FlinkRexUtil.isDeterministicInStreaming(

Review Comment:
   make sense to me



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {

Review Comment:
   ok



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {
+      val rightUniqueKeys = getUniqueKeysOfTemporalTable(join)
+
+      getJoinUniqueKeys(
+        join.joinType,
+        left.getRowType,
+        leftUniqueKeys,
+        rightUniqueKeys,
+        mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
+        rightUniqueKeys != null)
+    } else {
       null
-    )
+    }
+  }
+
+  private[flink] def getUniqueKeysOfTemporalTable(

Review Comment:
   The expected return values are different here, it'll be a little obscure to use 'JSet[ImmutableBitSet] != null' for representing lookupKeyContainsPrimaryKey of lookup join if unified to this method.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {

Review Comment:
   make sense



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1199297163

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "08b134e16f9fb779de5d30edb56b4a38fd63ba42",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "08b134e16f9fb779de5d30edb56b4a38fd63ba42",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08b134e16f9fb779de5d30edb56b4a38fd63ba42 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1202110810

   @godfreyhe the pr has been split, and when get this pr done, I'll update the separate pr of physical operator implementation  [FLINK-28568](https://issues.apache.org/jira/browse/FLINK-28568)


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] godfreyhe closed pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
godfreyhe closed pull request #20393: [FLINK-28570][table-planner]  Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error
URL: https://github.com/apache/flink/pull/20393


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1206049870

   @flinkbot run azure


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] godfreyhe commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r936415754


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+    val NO_PK = Array[Int]()

Review Comment:
   Array.empty[Int]



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val remainingConditionNonDeterministic =
+      join.remainingCondition.exists(c => !FlinkRexUtil.isDeterministicInStreaming(c))
+    lazy val calcOnTemporalTableNonDeterministic =
+      join.calcOnTemporalTable.exists(p => !FlinkRexUtil.isDeterministicInStreaming(p))
+
+    val rightUpsertKeys =
+      if (remainingConditionNonDeterministic || calcOnTemporalTableNonDeterministic) { null }

Review Comment:
   nit: 
   {
      null
   }



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala:
##########
@@ -98,6 +97,53 @@ class FlinkRelOptUtilTest {
         |         +- LogicalTableScan
       """.stripMargin
     assertEquals(expected2.trim, FlinkRelOptUtil.toString(rel, SqlExplainLevel.NO_ATTRIBUTES).trim)
+
+    // expect logical rel has no upsertKey info
+    assertEquals(
+      expected1.trim,
+      FlinkRelOptUtil.toString(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES, withUpsertKey = true).trim)
+  }
+
+  @Test
+  def testToStringWithUpsertKey(): Unit = {
+    val env = StreamExecutionEnvironment.createLocalEnvironment()
+    val tableEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING)
+
+    val table = env.fromElements[(Int, Long, String)]().toTable(tableEnv, 'a, 'b, 'c)
+    tableEnv.registerTable("MyTable", table)
+
+    val sqlQuery =
+      """
+        |WITH t1 AS (SELECT a, c, count(*) cnt FROM MyTable group by a, c),
+        |     t2 AS (SELECT a, max(b) b FROM MyTable WHERE b < 50 group by a)
+        |
+        |SELECT * FROM t1 JOIN t2 ON t1.a = t2.a
+      """.stripMargin
+    val result = tableEnv.sqlQuery(sqlQuery)
+    val rel = TableTestUtil.toRelNode(result)
+    val planner = tableEnv.asInstanceOf[TableEnvironmentImpl].getPlanner.asInstanceOf[PlannerBase]
+    // build optimized rel plan
+    val optimized = planner.optimize(rel)
+    val expected1 =
+      """
+        |Join(joinType=[InnerJoin], where=[=(a, a0)], select=[a, c, cnt, a0, b], leftInputSpec=[HasUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], upsertKeys=[[a, c, a0], [a, c]])
+        |:- Exchange(distribution=[hash[a]], upsertKeys=[[a, c]])
+        |:  +- GroupAggregate(groupBy=[a, c], select=[a, c, COUNT(*) AS cnt], upsertKeys=[[a, c]])
+        |:     +- Exchange(distribution=[hash[a, c]], upsertKeys=[])
+        |:        +- Calc(select=[a, c], upsertKeys=[])
+        |:           +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], upsertKeys=[])
+        |+- Exchange(distribution=[hash[a]], upsertKeys=[[a]])
+        |   +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b], upsertKeys=[[a]])
+        |      +- Exchange(distribution=[hash[a]], upsertKeys=[])
+        |         +- Calc(select=[a, b], where=[<(b, 50)], upsertKeys=[])
+        |            +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], upsertKeys=[])
+      """.stripMargin

Review Comment:
   put the plan result into a .out file ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);

Review Comment:
   if  the newChildren does not change, the `copy` can be avoid



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();

Review Comment:
   HashMap<>



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;

Review Comment:
   StreamPhysicalLegacySink<?>



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan

Review Comment:
   could StreamPhysicalDataStreamScan generate cdc event ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(

Review Comment:
   ditto



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+    val NO_PK = Array[Int]()
+    val outputPkIdx = if (temporalPkIdxs.isEmpty) {
+      NO_PK
+    } else {
+      calcOnTemporalTable match {
+        case Some(_) =>

Review Comment:
   Some(program)



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,24 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {
+      case streamRel: StreamPhysicalRel =>
+        val fmq = FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+        val upsertKeys = fmq.getUpsertKeys(streamRel)
+        if (null != upsertKeys && !upsertKeys.isEmpty) {
+          val fieldNames = streamRel.getRowType.getFieldNames
+          printValues.add(
+            Pair.of(
+              "upsertKeys",
+              upsertKeys
+                .map(bitset => s"[${bitset.toArray.map(fieldNames).mkString(", ")}]")
+                .mkString(", ")))
+        } else {
+          printValues.add(Pair.of("upsertKeys", ""))

Review Comment:
   only non-empty upsertKeys should be print ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicPhysicalPlanResolver.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.calcite.rel.RelNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * The {@link StreamNonDeterministicPhysicalPlanResolver} tries to resolve the correctness issue
+ * caused by 'Non-Deterministic Updates' (NDU) in a changelog pipeline. Changelog may contain kinds
+ * of message types: Insert (I), Delete (D), Update_before (UB), Update_after (UA).
+ *
+ * <p>There's no NDU problem in an insert-only changelog pipeline.
+ *
+ * <p>For the updates, there are two cases, with and without upsertKey(a metadata from {@link
+ * org.apache.flink.table.planner.plan.metadata.FlinkRelMdUpsertKeys} , consider it as the primary
+ * key of the changelog). The upsertKey can be always treated as deterministic, so if all the
+ * pipeline operators can transmit upsertKey normally (include working with sink's primary key),
+ * everything goes well.
+ *
+ * <p>The key issue is upsertKey can be easily lost in a pipeline or does not exist from the source
+ * or at the sink. All stateful operators can only process an update (D/UB/UA) message by comparing
+ * the complete row (retract by row) if without a key identifier, also include a sink without
+ * primary key definition that works as retractSink. So under the 'retract by row' mode, an stateful
+ * operator requires no non-deterministic column disturb the original changelog row. There are three
+ * killers:
+ *
+ * <p>1. Non-deterministic functions(include scalar, table, aggregate functions, builtin or custom
+ * ones)
+ *
+ * <p>2. LookupJoin on an evolving source
+ *
+ * <p>3. Cdc-source carries metadata fields(system columns, not belongs to the entity row itself)
+ *
+ * <p>For the first step, this resolver automatically enables the materialization for
+ * No.2(LookupJoin) if needed, and gives the detail error message for No.1 (Non-deterministic
+ * functions) and No.3(Cdc-source with metadata) which we think it is relatively easy to change the
+ * SQL(add materialization is not a good idea for now, it has very high cost and will bring too much
+ * complexity to the operators).
+ *
+ * <p>Why not do this validation and rewriting in physical-rewrite phase, like {@link
+ * org.apache.flink.table.planner.plan.optimize.program.FlinkChangelogModeInferenceProgram} does?
+ * Because the physical plan may be changed a lot after physical-rewrite being done, we should check
+ * the 'final' plan instead.
+ *
+ * <p>Some specific plan patterns:
+ *
+ * <p>1. Non-deterministic scalar function calls
+ *
+ * <pre>{@code
+ * Sink
+ *   |
+ * Project1{select col1,col2,now(),...}
+ *   |
+ * Scan1
+ * }</pre>
+ *
+ * <p>2. Non-deterministic table function calls
+ *
+ * <pre>{@code
+ *      Sink
+ *        |
+ *     Correlate
+ *     /      \
+ * Project1  TableFunctionScan1
+ *    |
+ *  Scan1
+ * }</pre>
+ *
+ * <p>3. lookup join: lookup a source which data may change over time
+ *
+ * <pre>{@code
+ *      Sink
+ *        |
+ *     LookupJoin
+ *     /      \
+ * Filter1  Source2
+ *    |
+ * Project1
+ *    |
+ *  Scan1
+ * }</pre>
+ *
+ * <p>3.1 lookup join: an inner project with non-deterministic function calls or remaining join
+ * condition is non-deterministic
+ *
+ * <pre>{@code
+ *      Sink
+ *        |
+ *     LookupJoin
+ *     /      \
+ * Filter1  Project2
+ *    |        |
+ * Project1   Source2
+ *    |
+ *  Scan1
+ * }</pre>
+ *
+ * <p>4. cdc source with metadata
+ *
+ * <pre>{@code
+ *     Sink
+ *       | no upsertKey can be inferred
+ *   Correlate
+ *     /      \
+ *   /       TableFunctionScan1(deterministic)
+ * Project1 {select id,name,attr1,op_time}
+ *   |
+ * Scan {cdc source <id,name,attr1,op_type,op_time> }
+ * }</pre>
+ *
+ * <p>4.1 cdc source with metadata
+ *
+ * <pre>{@code
+ *     Sink
+ *       | no upsertKey can be inferred
+ *   LookupJoin {lookup key not contains the lookup source's pk}
+ *     /      \
+ *   /       Source2
+ * Project1 {select id,name,attr1,op_time}
+ *   |
+ * Scan {cdc source <id,name,attr1,op_type,op_time> }
+ * }</pre>
+ *
+ * <p>CDC source with metadata is another form of non-deterministic update.
+ *
+ * <p>5. grouping keys with non-deterministic column
+ *
+ * <pre>{@code
+ * Sink{pk=(c3,day)}
+ *   | upsertKey=(c3,day)
+ *  GroupAgg{group by c3, day}
+ *   |
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...}
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ */
+public class StreamNonDeterministicPhysicalPlanResolver {
+
+    /**
+     * Try to resolve the NDU problem if configured {@link
+     * OptimizerConfigOptions#TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_HANDLING}. is in `TRY_RESOLVE`
+     * mode. Will raise an error if the NDU problems in the given plan can not be completely solved.
+     */
+    public static List<RelNode> resolvePhysicalPlan(
+            List<RelNode> expanded, TableConfig tableConfig) {
+        OptimizerConfigOptions.NonDeterministicUpdateHandling handling =
+                tableConfig
+                        .getConfiguration()
+                        .get(
+                                OptimizerConfigOptions
+                                        .TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_HANDLING);
+        if (handling == OptimizerConfigOptions.NonDeterministicUpdateHandling.TRY_RESOLVE) {
+            Preconditions.checkArgument(
+                    expanded.stream().allMatch(rel -> rel instanceof StreamPhysicalRel));
+            StreamNonDeterministicUpdatePlanVisitor planResolver =
+                    new StreamNonDeterministicUpdatePlanVisitor();
+            List<StreamPhysicalRel> physicalRelNodes =
+                    expanded.stream()
+                            .map(rel -> (StreamPhysicalRel) rel)
+                            .collect(Collectors.toList());
+            return physicalRelNodes.stream()
+                    .map(
+                            root ->
+                                    // set initial requirement to NO_REQUIRED_DETERMINISM
+                                    planResolver.visit(
+                                            root,
+                                            StreamNonDeterministicUpdatePlanVisitor
+                                                    .NO_REQUIRED_DETERMINISM))
+                    .collect(Collectors.toList());

Review Comment:
   they can be combined into one stream pipeline



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicPhysicalPlanResolver.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.calcite.rel.RelNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * The {@link StreamNonDeterministicPhysicalPlanResolver} tries to resolve the correctness issue
+ * caused by 'Non-Deterministic Updates' (NDU) in a changelog pipeline. Changelog may contain kinds
+ * of message types: Insert (I), Delete (D), Update_before (UB), Update_after (UA).
+ *
+ * <p>There's no NDU problem in an insert-only changelog pipeline.
+ *
+ * <p>For the updates, there are two cases, with and without upsertKey(a metadata from {@link
+ * org.apache.flink.table.planner.plan.metadata.FlinkRelMdUpsertKeys} , consider it as the primary
+ * key of the changelog). The upsertKey can be always treated as deterministic, so if all the
+ * pipeline operators can transmit upsertKey normally (include working with sink's primary key),
+ * everything goes well.
+ *
+ * <p>The key issue is upsertKey can be easily lost in a pipeline or does not exist from the source
+ * or at the sink. All stateful operators can only process an update (D/UB/UA) message by comparing
+ * the complete row (retract by row) if without a key identifier, also include a sink without
+ * primary key definition that works as retractSink. So under the 'retract by row' mode, an stateful

Review Comment:
   a stateful



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,24 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {
+      case streamRel: StreamPhysicalRel =>
+        val fmq = FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+        val upsertKeys = fmq.getUpsertKeys(streamRel)
+        if (null != upsertKeys && !upsertKeys.isEmpty) {
+          val fieldNames = streamRel.getRowType.getFieldNames
+          printValues.add(
+            Pair.of(
+              "upsertKeys",
+              upsertKeys
+                .map(bitset => s"[${bitset.toArray.map(fieldNames).mkString(", ")}]")

Review Comment:
   .mkString("[", ", ", "]")



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();

Review Comment:
   can be private



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(

Review Comment:
   we can provide a method: `StreamPhysicalRel visit(final StreamPhysicalRel rel) {}` whose the initial requirement is NO_REQUIRED_DETERMINISM 



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by

Review Comment:
   non determinism -> non-determinism result ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))

Review Comment:
                .map(program::expandLocalRef)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))

Review Comment:
            .filter(nonDeterministicCols::containsKey)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();

Review Comment:
   HashMap<>



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))

Review Comment:
      .filter(nonDeterministicOutput::containsKey)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));

Review Comment:
         Set<Integer> allRequiredInputSet = new HashSet<>(requireInputIndexes);



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))

Review Comment:
      .map(columns::indexOf)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());
+            int aggOutputIndex = inputFieldCnt;
+            for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                for (AggregateCall aggCall : groupSpec.getAggCalls()) {
+                    if (requireDeterminism.get(aggOutputIndex)) {
+                        requiredSourceInput(aggCall, allRequiredInputSet);
+                    }
+                    aggOutputIndex++;
+                }
+            }
+            assert allRequiredInputSet.size() <= inputFieldCnt;
+            return ImmutableBitSet.of(allRequiredInputSet.stream().collect(Collectors.toList()));
+        }
+    }
+
+    private void requiredSourceInput(
+            final AggregateCall aggCall, final Set<Integer> requiredInputSet) {
+        // add agg args first
+        aggCall.getArgList().forEach(arg -> requiredInputSet.add(arg));
+        // add agg filter args
+        if (aggCall.filterArg > -1) {
+            requiredInputSet.add(aggCall.filterArg);
+        }
+    }
+
+    private ImmutableBitSet requireDeterminismExcludeUpsertKey(
+            final StreamPhysicalRel inputRel, final ImmutableBitSet requireDeterminism) {
+        FlinkRelMetadataQuery fmq =
+                FlinkRelMetadataQuery.reuseOrCreate(inputRel.getCluster().getMetadataQuery());
+        Set<ImmutableBitSet> inputUpsertKeys = fmq.getUpsertKeys(inputRel);
+        ImmutableBitSet finalRequireDeterminism;
+        if (inputUpsertKeys == null || inputUpsertKeys.isEmpty()) {
+            finalRequireDeterminism = requireDeterminism;
+        } else {
+            if (inputUpsertKeys.stream().anyMatch(uk -> uk.contains(requireDeterminism))) {
+                // upsert keys can satisfy the requireDeterminism because they are always
+                // deterministic
+                finalRequireDeterminism = NO_REQUIRED_DETERMINISM;
+            } else {
+                // otherwise we should check the column(s) that not in upsert keys
+                List<ImmutableBitSet> leftKeys =
+                        inputUpsertKeys.stream()
+                                .map(uk -> requireDeterminism.except(uk))

Review Comment:
      .map(requireDeterminism::except)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));

Review Comment:
   .forEach(allRequiredInputSet::add);



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());

Review Comment:
     overSpec.getGroups().forEach(OverSpec.GroupSpec::getAggCalls);



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());
+            int aggOutputIndex = inputFieldCnt;
+            for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                for (AggregateCall aggCall : groupSpec.getAggCalls()) {
+                    if (requireDeterminism.get(aggOutputIndex)) {
+                        requiredSourceInput(aggCall, allRequiredInputSet);
+                    }
+                    aggOutputIndex++;
+                }
+            }
+            assert allRequiredInputSet.size() <= inputFieldCnt;
+            return ImmutableBitSet.of(allRequiredInputSet.stream().collect(Collectors.toList()));

Review Comment:
   ImmutableBitSet.of(new ArrayList<>(allRequiredInputSet));



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());
+            int aggOutputIndex = inputFieldCnt;
+            for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                for (AggregateCall aggCall : groupSpec.getAggCalls()) {
+                    if (requireDeterminism.get(aggOutputIndex)) {
+                        requiredSourceInput(aggCall, allRequiredInputSet);
+                    }
+                    aggOutputIndex++;
+                }
+            }
+            assert allRequiredInputSet.size() <= inputFieldCnt;
+            return ImmutableBitSet.of(allRequiredInputSet.stream().collect(Collectors.toList()));
+        }
+    }
+
+    private void requiredSourceInput(
+            final AggregateCall aggCall, final Set<Integer> requiredInputSet) {
+        // add agg args first
+        aggCall.getArgList().forEach(arg -> requiredInputSet.add(arg));
+        // add agg filter args
+        if (aggCall.filterArg > -1) {
+            requiredInputSet.add(aggCall.filterArg);
+        }
+    }
+
+    private ImmutableBitSet requireDeterminismExcludeUpsertKey(
+            final StreamPhysicalRel inputRel, final ImmutableBitSet requireDeterminism) {
+        FlinkRelMetadataQuery fmq =
+                FlinkRelMetadataQuery.reuseOrCreate(inputRel.getCluster().getMetadataQuery());
+        Set<ImmutableBitSet> inputUpsertKeys = fmq.getUpsertKeys(inputRel);
+        ImmutableBitSet finalRequireDeterminism;
+        if (inputUpsertKeys == null || inputUpsertKeys.isEmpty()) {
+            finalRequireDeterminism = requireDeterminism;
+        } else {
+            if (inputUpsertKeys.stream().anyMatch(uk -> uk.contains(requireDeterminism))) {
+                // upsert keys can satisfy the requireDeterminism because they are always
+                // deterministic
+                finalRequireDeterminism = NO_REQUIRED_DETERMINISM;
+            } else {
+                // otherwise we should check the column(s) that not in upsert keys
+                List<ImmutableBitSet> leftKeys =
+                        inputUpsertKeys.stream()
+                                .map(uk -> requireDeterminism.except(uk))
+                                .collect(Collectors.toList());
+                if (leftKeys.isEmpty()) {
+                    finalRequireDeterminism = NO_REQUIRED_DETERMINISM;
+                } else {
+                    leftKeys.sort((f, f1) -> f.cardinality() - f.cardinality());

Review Comment:
   nit: f1 -> ignored ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {

Review Comment:
   extract some basic method with FlinkRexUtil.isDeterministicInStreaming 



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());
+            int aggOutputIndex = inputFieldCnt;
+            for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                for (AggregateCall aggCall : groupSpec.getAggCalls()) {
+                    if (requireDeterminism.get(aggOutputIndex)) {
+                        requiredSourceInput(aggCall, allRequiredInputSet);
+                    }
+                    aggOutputIndex++;
+                }
+            }
+            assert allRequiredInputSet.size() <= inputFieldCnt;
+            return ImmutableBitSet.of(allRequiredInputSet.stream().collect(Collectors.toList()));
+        }
+    }
+
+    private void requiredSourceInput(
+            final AggregateCall aggCall, final Set<Integer> requiredInputSet) {
+        // add agg args first
+        aggCall.getArgList().forEach(arg -> requiredInputSet.add(arg));

Review Comment:
    requiredInputSet.addAll(aggCall.getArgList());



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {

Review Comment:
   can we handle the `inputInsertOnly ` at the begin of the visit method ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(

Review Comment:
   throw TableException directly in this method and rename it throwNonDeterministicConditionError ?
   



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx

Review Comment:
   outputPkIdx won't be null



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {

Review Comment:
   nit: we can use `JavaScalaConversionUtil.toJava(lookupJoin.calcOnTemporalTable())
                           .ifPresent(
                                   c ->
                                           checkNonDeterministicRexProgram(
                                                   requireDeterminism, c, lookupJoin));`
   to avoid if branch



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));

Review Comment:
   .allMatch(outputPkBitSet::get);



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0

Review Comment:
   `outputPkIdx.length > 0` is necessary ?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {
+      val rightUniqueKeys = getUniqueKeysOfTemporalTable(join)
+
+      getJoinUniqueKeys(
+        join.joinType,
+        left.getRowType,
+        leftUniqueKeys,
+        rightUniqueKeys,
+        mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
+        rightUniqueKeys != null)
+    } else {
       null
-    )
+    }
+  }
+
+  private[flink] def getUniqueKeysOfTemporalTable(

Review Comment:
   CommonPhysicalLookupJoin#lookupKeyContainsPrimaryKey uses `getUniqueKeysOfTemporalTable ` ?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())

Review Comment:
    .map(Column::getName)



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala:
##########
@@ -62,14 +62,19 @@ class StreamPhysicalIncrementalGroupAggregate(
     inputRel: RelNode,
     val partialAggGrouping: Array[Int],
     val partialAggCalls: Array[AggregateCall],
-    val finalAggGrouping: Array[Int],

Review Comment:
   make sense



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {

Review Comment:
   The method name may be misleading, because the method may be treat as get pk for lookup join 



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source

Review Comment:
    metadata 



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalJoin.scala:
##########
@@ -80,4 +81,14 @@ abstract class CommonPhysicalJoin(
       .item("select", getRowType.getFieldNames.mkString(", "))
   }
 
+  def getUniqueKeys(input: RelNode, keys: Array[Int]): List[Array[Int]] = {

Review Comment:
   https://issues.apache.org/jira/browse/FLINK-28787, we can add a TODO before the method



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()

Review Comment:
          rank.partitionKey().toList().forEach(bitSetBuilder::set);



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {
+        val program = calcOnTemporalTable.get
+        val outputMapping = program.getProjectList.asScala.zipWithIndex
+          .map { case (ref, index) => (index, program.expandLocalRef(ref)) }
+          .map {
+            case (outIndex, ref) =>
+              ref match {
+                case inputRef: RexInputRef => (inputRef.getIndex, outIndex)
+                case _ => (-1, -1)
+              }
+          }
+          .toMap
+        val outputPk = temporalPkIdxs.forall(outputMapping.contains)
+        if (outputPk) {
+          // remapping pk index
+          temporalPkIdxs.map(outputMapping.get(_).get)
+        } else {
+          Array[Int]()
+        }
+      } else {
+        temporalPkIdxs
+      }
+    } else {
+      // temporal table has no pk, no uk produces
+      Array[Int]()
+    }
+
+    outputPkIdx
+  }
+
+  private def getPrimaryKeyIndexesOfTemporalTable: Array[Int] = {
+    // get primary key columns of lookup table if exists
+    val pkColumns = getPrimaryKeyColumnsOfTemporalTable
+    if (pkColumns.isDefined) {
+      val newSchema = temporalTable.getRowType.getFieldNames
+      pkColumns.get.toArray().map(col => newSchema.indexOf(col))
+    } else {
+      Array[Int]()
+    }
+  }
+
+  private def getPrimaryKeyColumnsOfTemporalTable: Option[util.List[String]] = {
+    val schema = temporalTable match {
+      case t: TableSourceTable =>
+        TableSchema.fromResolvedSchema(t.contextResolvedTable.getResolvedSchema)

Review Comment:
   another approach: for TableSourceTable and IntermediateRelTable, we use ResolvedSchema to get pk, while for LegacyTableSourceTable, we use TableSchema to get pk. If LegacyTableSourceTable is removed in the future, only related line will be deleted.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize

Review Comment:
   StreamPhysicalChangelogNormalize also have upsert keys



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r936682098


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {
+      val rightUniqueKeys = getUniqueKeysOfTemporalTable(join)
+
+      getJoinUniqueKeys(
+        join.joinType,
+        left.getRowType,
+        leftUniqueKeys,
+        rightUniqueKeys,
+        mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
+        rightUniqueKeys != null)
+    } else {
       null
-    )
+    }
+  }
+
+  private[flink] def getUniqueKeysOfTemporalTable(

Review Comment:
   I almost forgot an important difference here: the  lookupKeys directly from joinInfo is not always equivalent to CommonPhysicalLookupJoin#allLookupKeys
   e.g.,  an inner lookup join condition: 'left.id = dim.id and dim.ver = 1'
   after optimization, the joinInfo only contains 'left.id = dim.id', and 'dim.ver = 1' will be a local predicate on temporal table (stored in calcOnTemporalTable as a rexprogram), the CommonPhysicalLookupJoin#allLookupKeys will consider this 'dim.ver = 1' be a constant lookup key candidate.
   



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java:
##########
@@ -142,4 +142,33 @@ public class OptimizerConfigOptions {
                     .withDescription(
                             "When it is true, the optimizer will merge the operators with pipelined shuffling "
                                     + "into a multiple input operator to reduce shuffling and improve performance. Default value is true.");
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<NonDeterministicUpdateHandling>
+            TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_HANDLING =
+                    key("table.optimizer.non-deterministic-update.handling")

Review Comment:
   As discussed offline, we'll change this to 'table.optimizer.non-deterministic-update.strategy'



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {

Review Comment:
   'get pk for lookup join' is exactly the purpose here...
   how about 'getOutputIndexesOfTemporalTablePrimaryKey'?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {

Review Comment:
   for batch scenario, a dynamic function should be query wise(not record wise), once the query was submited/executed, the value of the function will be determined, while in streaming both functions are record wise (evaluated per record)



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala:
##########
@@ -488,16 +488,36 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
       ignoreNulls: Boolean): util.Set[ImmutableBitSet] = {
     val left = join.getInput
     val leftUniqueKeys = mq.getUniqueKeys(left, ignoreNulls)
-    val leftType = left.getRowType
-    getJoinUniqueKeys(
-      join.joinType,
-      leftType,
-      leftUniqueKeys,
-      null,
-      mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
-      // TODO get uniqueKeys from TableSchema of TableSource
+
+    if (leftUniqueKeys != null) {
+      val rightUniqueKeys = getUniqueKeysOfTemporalTable(join)
+
+      getJoinUniqueKeys(
+        join.joinType,
+        left.getRowType,
+        leftUniqueKeys,
+        rightUniqueKeys,
+        mq.areColumnsUnique(left, join.joinInfo.leftSet, ignoreNulls),
+        rightUniqueKeys != null)
+    } else {
       null
-    )
+    }
+  }
+
+  private[flink] def getUniqueKeysOfTemporalTable(

Review Comment:
   So I add the method comment "Check if lookup key contains primary key, include constant lookup keys. "



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala:
##########
@@ -98,6 +97,53 @@ class FlinkRelOptUtilTest {
         |         +- LogicalTableScan
       """.stripMargin
     assertEquals(expected2.trim, FlinkRelOptUtil.toString(rel, SqlExplainLevel.NO_ATTRIBUTES).trim)
+
+    // expect logical rel has no upsertKey info
+    assertEquals(
+      expected1.trim,
+      FlinkRelOptUtil.toString(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES, withUpsertKey = true).trim)
+  }
+
+  @Test
+  def testToStringWithUpsertKey(): Unit = {
+    val env = StreamExecutionEnvironment.createLocalEnvironment()
+    val tableEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING)
+
+    val table = env.fromElements[(Int, Long, String)]().toTable(tableEnv, 'a, 'b, 'c)
+    tableEnv.registerTable("MyTable", table)
+
+    val sqlQuery =
+      """
+        |WITH t1 AS (SELECT a, c, count(*) cnt FROM MyTable group by a, c),
+        |     t2 AS (SELECT a, max(b) b FROM MyTable WHERE b < 50 group by a)
+        |
+        |SELECT * FROM t1 JOIN t2 ON t1.a = t2.a
+      """.stripMargin
+    val result = tableEnv.sqlQuery(sqlQuery)
+    val rel = TableTestUtil.toRelNode(result)
+    val planner = tableEnv.asInstanceOf[TableEnvironmentImpl].getPlanner.asInstanceOf[PlannerBase]
+    // build optimized rel plan
+    val optimized = planner.optimize(rel)
+    val expected1 =
+      """
+        |Join(joinType=[InnerJoin], where=[=(a, a0)], select=[a, c, cnt, a0, b], leftInputSpec=[HasUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], upsertKeys=[[a, c, a0], [a, c]])
+        |:- Exchange(distribution=[hash[a]], upsertKeys=[[a, c]])
+        |:  +- GroupAggregate(groupBy=[a, c], select=[a, c, COUNT(*) AS cnt], upsertKeys=[[a, c]])
+        |:     +- Exchange(distribution=[hash[a, c]], upsertKeys=[])
+        |:        +- Calc(select=[a, c], upsertKeys=[])
+        |:           +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], upsertKeys=[])
+        |+- Exchange(distribution=[hash[a]], upsertKeys=[[a]])
+        |   +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b], upsertKeys=[[a]])
+        |      +- Exchange(distribution=[hash[a]], upsertKeys=[])
+        |         +- Calc(select=[a, b], where=[<(b, 50)], upsertKeys=[])
+        |            +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], upsertKeys=[])
+      """.stripMargin

Review Comment:
   maybe better let it done for the all tests of FlinkRelOptUtilTest (do refact: let it extends TableTestBase)?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by

Review Comment:
   updated to : non-deterministic result



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeys.scala:
##########
@@ -228,17 +229,29 @@ class FlinkRelMdUpsertKeys private extends MetadataHandler[UpsertKeys] {
       join: CommonPhysicalLookupJoin,
       mq: RelMetadataQuery): util.Set[ImmutableBitSet] = {
     val left = join.getInput
-    val leftKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
     val leftType = left.getRowType
     val leftJoinKeys = join.joinInfo.leftSet
+    // differs from regular join, here we do not filterKeys because there's no shuffle on join keys
+    // by default.
+    val leftUpsertKeys = FlinkRelMetadataQuery.reuseOrCreate(mq).getUpsertKeys(left)
+    val rightUniqueKeys = FlinkRelMdUniqueKeys.INSTANCE.getUniqueKeysOfTemporalTable(join)
+
+    val remainingConditionNonDeterministic =
+      join.remainingCondition.exists(c => !FlinkRexUtil.isDeterministicInStreaming(c))
+    lazy val calcOnTemporalTableNonDeterministic =
+      join.calcOnTemporalTable.exists(p => !FlinkRexUtil.isDeterministicInStreaming(p))
+
+    val rightUpsertKeys =
+      if (remainingConditionNonDeterministic || calcOnTemporalTableNonDeterministic) { null }

Review Comment:
   the auto-format takes effect when I haven't given enough enter lines..  
   now get done for this 



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);

Review Comment:
   the cost of 'copy' is not too much higher compared to 'compare'?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;

Review Comment:
   ok



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {
+        try {
+            RexVisitor<Void> visitor =
+                    new RexVisitorImpl<Void>(true) {
+                        public Void visitCall(RexCall call) {
+                            // dynamic function call is also non-deterministic to streaming
+                            if (!call.getOperator().isDeterministic()
+                                    || call.getOperator().isDynamicFunction()) {
+                                throw new Util.FoundOne(call.getOperator().getName());
+                            } else {
+                                return super.visitCall(call);
+                            }
+                        }
+                    };
+            e.accept(visitor);
+            return Optional.empty();
+        } catch (Util.FoundOne ex) {
+            Util.swallow(ex, null);
+            return Optional.ofNullable(ex.getNode().toString());
+        }
+    }
+
+    private ImmutableBitSet mappingRequireDeterminismToInput(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalOverAggregateBase overAgg) {
+        int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+        List<Integer> requireInputIndexes =
+                requireDeterminism.toList().stream()
+                        .filter(index -> index < inputFieldCnt)
+                        .collect(Collectors.toList());
+        if (requireInputIndexes.size() == inputFieldCnt) {
+            return ImmutableBitSet.range(inputFieldCnt);
+        } else {
+            Set<Integer> allRequiredInputSet = new HashSet<>();
+            requireInputIndexes.forEach(index -> allRequiredInputSet.add(index));
+
+            OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+            // add partitionKeys
+            Arrays.stream(overSpec.getPartition().getFieldIndices())
+                    .forEach(index -> allRequiredInputSet.add(index));
+            // add aggCall's input
+            overSpec.getGroups().forEach(group -> group.getAggCalls());
+            int aggOutputIndex = inputFieldCnt;
+            for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                for (AggregateCall aggCall : groupSpec.getAggCalls()) {
+                    if (requireDeterminism.get(aggOutputIndex)) {
+                        requiredSourceInput(aggCall, allRequiredInputSet);
+                    }
+                    aggOutputIndex++;
+                }
+            }
+            assert allRequiredInputSet.size() <= inputFieldCnt;
+            return ImmutableBitSet.of(allRequiredInputSet.stream().collect(Collectors.toList()));
+        }
+    }
+
+    private void requiredSourceInput(
+            final AggregateCall aggCall, final Set<Integer> requiredInputSet) {
+        // add agg args first
+        aggCall.getArgList().forEach(arg -> requiredInputSet.add(arg));
+        // add agg filter args
+        if (aggCall.filterArg > -1) {
+            requiredInputSet.add(aggCall.filterArg);
+        }
+    }
+
+    private ImmutableBitSet requireDeterminismExcludeUpsertKey(
+            final StreamPhysicalRel inputRel, final ImmutableBitSet requireDeterminism) {
+        FlinkRelMetadataQuery fmq =
+                FlinkRelMetadataQuery.reuseOrCreate(inputRel.getCluster().getMetadataQuery());
+        Set<ImmutableBitSet> inputUpsertKeys = fmq.getUpsertKeys(inputRel);
+        ImmutableBitSet finalRequireDeterminism;
+        if (inputUpsertKeys == null || inputUpsertKeys.isEmpty()) {
+            finalRequireDeterminism = requireDeterminism;
+        } else {
+            if (inputUpsertKeys.stream().anyMatch(uk -> uk.contains(requireDeterminism))) {
+                // upsert keys can satisfy the requireDeterminism because they are always
+                // deterministic
+                finalRequireDeterminism = NO_REQUIRED_DETERMINISM;
+            } else {
+                // otherwise we should check the column(s) that not in upsert keys
+                List<ImmutableBitSet> leftKeys =
+                        inputUpsertKeys.stream()
+                                .map(uk -> requireDeterminism.except(uk))
+                                .collect(Collectors.toList());
+                if (leftKeys.isEmpty()) {
+                    finalRequireDeterminism = NO_REQUIRED_DETERMINISM;
+                } else {
+                    leftKeys.sort((f, f1) -> f.cardinality() - f.cardinality());

Review Comment:
   changed to 'Comparator.comparingInt(ImmutableBitSet::cardinality)'



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0

Review Comment:
   you're right, can be removed.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {

Review Comment:
   There're different logic for groupAgg, windowAgg, overAgg, regular join and also source node, not all these process are the same, so I tend to keep it is for now



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source

Review Comment:
   ok



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala:
##########
@@ -85,6 +87,24 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withUpsertKey) rel match {
+      case streamRel: StreamPhysicalRel =>
+        val fmq = FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+        val upsertKeys = fmq.getUpsertKeys(streamRel)
+        if (null != upsertKeys && !upsertKeys.isEmpty) {
+          val fieldNames = streamRel.getRowType.getFieldNames
+          printValues.add(
+            Pair.of(
+              "upsertKeys",
+              upsertKeys
+                .map(bitset => s"[${bitset.toArray.map(fieldNames).mkString(", ")}]")
+                .mkString(", ")))
+        } else {
+          printValues.add(Pair.of("upsertKeys", ""))

Review Comment:
   ok



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLookupJoin.scala:
##########
@@ -279,6 +280,83 @@ abstract class CommonPhysicalLookupJoin(
     constantLookupKeys.toMap[Int, LookupKey] ++ fieldRefLookupKeys.toMap[Int, LookupKey]
   }
 
+  /** Check if lookup key contains primary key, include constant lookup keys. */
+  def lookupKeyContainsPrimaryKey(): Boolean = {
+    val outputPkIdx = getOutputPrimaryKeyIndexes
+    // use allLookupKeys instead of joinInfo.rightSet because there may exists constant
+    // lookup key(s) which are not included in joinInfo.rightKeys.
+    outputPkIdx.nonEmpty && outputPkIdx.forall(index => allLookupKeys.contains(index))
+  }
+
+  /** Get final output pk indexes if exists, otherwise will get empty. */
+  def getOutputPrimaryKeyIndexes: Array[Int] = {
+    val temporalPkIdxs = getPrimaryKeyIndexesOfTemporalTable
+
+    val outputPkIdx = if (!temporalPkIdxs.isEmpty) {
+      if (calcOnTemporalTable.isDefined) {
+        val program = calcOnTemporalTable.get
+        val outputMapping = program.getProjectList.asScala.zipWithIndex
+          .map { case (ref, index) => (index, program.expandLocalRef(ref)) }
+          .map {
+            case (outIndex, ref) =>
+              ref match {
+                case inputRef: RexInputRef => (inputRef.getIndex, outIndex)
+                case _ => (-1, -1)
+              }
+          }
+          .toMap
+        val outputPk = temporalPkIdxs.forall(outputMapping.contains)
+        if (outputPk) {
+          // remapping pk index
+          temporalPkIdxs.map(outputMapping.get(_).get)
+        } else {
+          Array[Int]()
+        }
+      } else {
+        temporalPkIdxs
+      }
+    } else {
+      // temporal table has no pk, no uk produces
+      Array[Int]()
+    }
+
+    outputPkIdx
+  }
+
+  private def getPrimaryKeyIndexesOfTemporalTable: Array[Int] = {
+    // get primary key columns of lookup table if exists
+    val pkColumns = getPrimaryKeyColumnsOfTemporalTable
+    if (pkColumns.isDefined) {
+      val newSchema = temporalTable.getRowType.getFieldNames
+      pkColumns.get.toArray().map(col => newSchema.indexOf(col))
+    } else {
+      Array[Int]()
+    }
+  }
+
+  private def getPrimaryKeyColumnsOfTemporalTable: Option[util.List[String]] = {
+    val schema = temporalTable match {
+      case t: TableSourceTable =>
+        TableSchema.fromResolvedSchema(t.contextResolvedTable.getResolvedSchema)

Review Comment:
   Ok



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala:
##########
@@ -585,6 +585,50 @@ object FlinkRexUtil {
     }
     false
   }
+
+  /**
+   * Returns whether a given expression is deterministic in streaming scenario, differs from
+   * calcite's [[RexUtil]], it considers both non-deterministic and dynamic functions.
+   */
+  def isDeterministicInStreaming(e: RexNode): Boolean = try {

Review Comment:
   for batch scenario, a dynamic function should be query wise(not record wise), once the query was submited/executed, the value of the function will be determined, while in streaming both functions are record wise (evaluated per record)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan

Review Comment:
   theoretically yes, but we can not detect it for now, so I keep consistent with FlinkChangelogModeInferenceProgram
   
   ```
         case _: StreamPhysicalDataStreamScan | _: StreamPhysicalLegacyTableSourceScan |
             _: StreamPhysicalValues =>
           // DataStream, TableSource and Values only support producing insert-only messages
           createNewNode(rel, List(), ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester)
   
   ```



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(
+            final String ndCall, final RexNode condition, final StreamPhysicalRel relatedRel) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append(
+                String.format(NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE, ndCall, condition));
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    private String generateNonDeterministicColumnsErrorMessage(
+            final List<Integer> indexes,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel,
+            final Map<Integer, String> ndCallMap,
+            final Optional<String> ndCallName) {
+        StringBuilder errorMsg = new StringBuilder();
+        errorMsg.append("The column(s): ");
+        int index = 0;
+        for (String column : rowType.getFieldNames()) {
+            if (indexes.contains(index)) {
+                errorMsg.append(column).append("(generated by non-deterministic function: ");
+                if (ndCallName.isPresent()) {
+                    errorMsg.append(ndCallName.get());
+                } else {
+                    errorMsg.append(ndCallMap.get(index));
+                }
+                errorMsg.append(" ) ");
+            }
+            index++;
+        }
+        errorMsg.append(
+                "can not satisfy the determinism requirement for correctly processing update message("
+                        + "'UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has"
+                        + " no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update "
+                        + "messages. Please consider removing these non-deterministic columns or making them "
+                        + "deterministic by using deterministic functions.\n");
+        errorMsg.append("\nrelated rel plan:\n")
+                .append(
+                        FlinkRelOptUtil.toString(
+                                relatedRel,
+                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                false,
+                                true,
+                                false,
+                                true));
+
+        return errorMsg.toString();
+    }
+
+    /**
+     * Returns the non-deterministic call name from the given expression, differs from calcite's
+     * [[RexUtil]], it considers both non-deterministic and dynamic functions.
+     */
+    private Optional<String> getNonDeterministicCallName(final RexNode e) {

Review Comment:
   done. extract an new FlinkRexUtil#getNonDeterministicCallNameInStreaming



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize
+                || rel instanceof StreamPhysicalDropUpdateBefore
+                || rel instanceof StreamPhysicalMiniBatchAssigner
+                || rel instanceof StreamPhysicalUnion
+                || rel instanceof StreamPhysicalSort
+                || rel instanceof StreamPhysicalLimit
+                || rel instanceof StreamPhysicalSortLimit
+                || rel instanceof StreamPhysicalTemporalSort
+                || rel instanceof StreamPhysicalWatermarkAssigner
+                || rel instanceof StreamPhysicalExchange) {
+            // transit requireDeterminism transparently
+            return transmitDeterminismRequirement(rel, requireDeterminism);
+        } else if (rel instanceof StreamPhysicalMatch) {
+            // TODO to be supported in FLINK-28743
+            throw new TableException(
+                    "Unsupported to resolve non-deterministic issue in match-recognize.");
+        } else {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "Unsupported to visit node %s, please add the visit implementation if it is a newly added stream physical node.",
+                            rel.getClass().getSimpleName()));
+        }
+    }
+
+    // helper methods
+    private boolean inputInsertOnly(final StreamPhysicalRel rel) {
+        return ChangelogPlanUtils.inputInsertOnly(rel);
+    }
+
+    private StreamPhysicalRel transmitDeterminismRequirement(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = visitInputs(parent, requireDeterminism);
+        return (StreamPhysicalRel) parent.copy(parent.getTraitSet(), newChildren);
+    }
+
+    private List<RelNode> visitInputs(
+            final StreamPhysicalRel parent, final ImmutableBitSet requireDeterminism) {
+        List<RelNode> newChildren = new ArrayList<>();
+        for (int index = 0; index < parent.getInputs().size(); index++) {
+            StreamPhysicalRel input = (StreamPhysicalRel) parent.getInput(index);
+            // unified processing on input upsertKey
+            newChildren.add(
+                    visit(input, requireDeterminismExcludeUpsertKey(input, requireDeterminism)));
+        }
+        return newChildren;
+    }
+
+    private StreamPhysicalRel visitJoinChild(
+            final ImmutableBitSet requireDeterminism,
+            final StreamPhysicalRel rel,
+            final boolean inputHasUpdate,
+            final int leftFieldCnt,
+            final boolean isLeft,
+            final int[] joinKeys,
+            final List<int[]> inputUniqueKeys) {
+        JoinInputSideSpec joinInputSideSpec =
+                JoinUtil.analyzeJoinInput(
+                        ShortcutUtils.unwrapClassLoader(rel),
+                        InternalTypeInfo.of(FlinkTypeFactory.toLogicalRowType(rel.getRowType())),
+                        joinKeys,
+                        inputUniqueKeys);
+        ImmutableBitSet inputRequireDeterminism;
+        if (inputHasUpdate) {
+            if (joinInputSideSpec.hasUniqueKey() || joinInputSideSpec.joinKeyContainsUniqueKey()) {
+                // join hasUniqueKey or joinKeyContainsUniqueKey, then transmit corresponding
+                // requirement to input
+                if (isLeft) {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index < leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                } else {
+                    inputRequireDeterminism =
+                            ImmutableBitSet.of(
+                                    requireDeterminism.toList().stream()
+                                            .filter(index -> index >= leftFieldCnt)
+                                            .map(index -> index - leftFieldCnt)
+                                            .collect(Collectors.toList()));
+                }
+            } else {
+                // join need to retract by whole input row
+                inputRequireDeterminism = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+            }
+        } else {
+            inputRequireDeterminism = NO_REQUIRED_DETERMINISM;
+        }
+        return transmitDeterminismRequirement(rel, inputRequireDeterminism);
+    }
+
+    /** Extracts the out from source field index mapping of the given projects. */
+    private Map<Integer, Integer> extractSourceMapping(final List<RexNode> projects) {
+        Map<Integer, Integer> mapOutFromInPos = new HashMap();
+
+        for (int index = 0; index < projects.size(); index++) {
+            RexNode expr = projects.get(index);
+            if (expr instanceof RexInputRef) {
+                mapOutFromInPos.put(index, ((RexInputRef) expr).getIndex());
+            } else if (expr instanceof RexCall) {
+                // rename or cast call
+                RexCall call = (RexCall) expr;
+                if ((call.getKind().equals(SqlKind.AS) || call.getKind().equals(SqlKind.CAST))
+                        && (call.getOperands().get(0) instanceof RexInputRef)) {
+                    RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+                    mapOutFromInPos.put(index, ref.getIndex());
+                }
+            } else if (expr instanceof RexLiteral) {
+                mapOutFromInPos.put(index, -1);
+            }
+            // else ignore
+        }
+
+        return mapOutFromInPos;
+    }
+
+    private void checkNonDeterministicRexProgram(
+            final ImmutableBitSet requireDeterminism,
+            final RexProgram program,
+            final StreamPhysicalRel relatedRel) {
+        if (null != program.getCondition()) {
+            // firstly check if exists non-deterministic condition
+            RexNode rexNode = program.expandLocalRef(program.getCondition());
+            checkNonDeterministicCondition(rexNode, relatedRel);
+        }
+        // extract all non-deterministic output columns first and check if any of them were
+        // required be deterministic.
+        List<RexNode> projects =
+                program.getProjectList().stream()
+                        .map(expr -> program.expandLocalRef(expr))
+                        .collect(Collectors.toList());
+        Map<Integer, String> nonDeterministicCols = new HashMap<>();
+        for (int index = 0; index < projects.size(); index++) {
+            Optional<String> ndCall = getNonDeterministicCallName(projects.get(index));
+            if (ndCall.isPresent()) {
+                nonDeterministicCols.put(index, ndCall.get());
+            } // else ignore
+        }
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicCols.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            relatedRel.getRowType(),
+                            relatedRel,
+                            nonDeterministicCols,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private void checkNonDeterministicCondition(
+            final RexNode condition, final StreamPhysicalRel relatedRel) {
+        Optional<String> ndCall = getNonDeterministicCallName(condition);
+        if (ndCall.isPresent()) {
+            throw new TableException(
+                    generateNonDeterministicConditionErrorMessage(
+                            ndCall.get(), condition, relatedRel));
+        }
+    }
+
+    private void checkUnsatisfiedDeterminism(
+            final ImmutableBitSet requireDeterminism,
+            final int aggStartIndex,
+            final List<AggregateCall> aggCalls,
+            final RelDataType rowType,
+            final StreamPhysicalRel relatedRel) {
+        Map<Integer, String> nonDeterministicOutput = new HashMap();
+        // skip checking non-deterministic columns in grouping keys or filter args in agg call
+        // because they were pushed down to input project which processes input only message
+        int aggOutputIndex = aggStartIndex;
+        for (AggregateCall aggCall : aggCalls) {
+            if (!aggCall.getAggregation().isDeterministic()
+                    || aggCall.getAggregation().isDynamicFunction()) {
+                nonDeterministicOutput.put(aggOutputIndex, aggCall.getAggregation().getName());
+            }
+            aggOutputIndex++;
+        }
+        // check if exist non-deterministic aggCalls which were in requireDeterminism
+        List<Integer> unsatisfiedColumns =
+                requireDeterminism.toList().stream()
+                        .filter(index -> nonDeterministicOutput.containsKey(index))
+                        .collect(Collectors.toList());
+        if (!unsatisfiedColumns.isEmpty()) {
+            String errorMsg =
+                    generateNonDeterministicColumnsErrorMessage(
+                            unsatisfiedColumns,
+                            rowType,
+                            relatedRel,
+                            nonDeterministicOutput,
+                            Optional.empty());
+            throw new TableException(errorMsg);
+        }
+    }
+
+    private String generateNonDeterministicConditionErrorMessage(

Review Comment:
   good, will update it



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize

Review Comment:
   StreamPhysicalChangelogNormalize is a special op here, it is generated by planner for  upsert source only, it must have upsert keys that equal to the source's pk, so here can just simply do transmit.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.planner.plan.optimize;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.constraints.UniqueConstraint;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregateBase;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank;
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.plan.utils.JoinUtil;
+import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An inner visitor to validate if there's any NDU problems which may cause wrong result and try to
+ * rewrite lookup join node with materialization (to eliminate the non determinism generated by
+ * lookup join node only).
+ *
+ * <p>The visitor will try to satisfy the required determinism(represent by ImmutableBitSet) from
+ * root. The transmission rule of required determinism:
+ *
+ * <p>0. all required determinism is under the precondition: input has updates, that is say no
+ * update determinism will be passed to an insert only stream
+ *
+ * <p>1. the initial required determinism to the root node(e.g., sink node) was none
+ *
+ * <p>2. for a relNode, it will process on two aspects: - can satisfy non-empty required determinism
+ * - actively requires determinism from input by self requirements(e.g., stateful node works on
+ * retract by row mode)
+ *
+ * <pre>{@code
+ * Rel3
+ *  | require input
+ *  v
+ * Rel2 {1. satisfy Rel3's requirement 2. append new requirement to input Rel1}
+ *  | require input
+ *  v
+ * Rel1
+ * }</pre>
+ *
+ * <p>the requiredDeterminism passed to input will exclude columns which were in upsertKey, e.g.,
+ *
+ * <pre>{@code
+ * Sink {pk=(c3)} requiredDeterminism=(c3)
+ *   | passed requiredDeterminism={}
+ *  GroupAgg{group by c3, day} append requiredDeterminism=(c3, day)
+ *   | passed requiredDeterminism=(c3, day)
+ * Project{select c1,c2,DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') day,...} [x] can not satisfy
+ *   |
+ * Deduplicate{keep last row, dedup on c1,c2}
+ *   |
+ *  Scan
+ * }</pre>
+ *
+ * <p>3. for a sink node, it will require key columns' determinism when primary key is defined or
+ * require all columns' determinism when no primary key is defined
+ *
+ * <p>4. for a cdc source node(which will generate updates), the metadata columns are treated as
+ * non-deterministic.
+ */
+public class StreamNonDeterministicUpdatePlanVisitor {
+    public static final ImmutableBitSet NO_REQUIRED_DETERMINISM = ImmutableBitSet.of();
+
+    public static final String NON_DETERMINISTIC_CONDITION_ERROR_MSG_TEMPLATE =
+            "There exists non deterministic function: '%s' in condition: '%s' which may cause wrong result in update pipeline.";
+
+    public StreamPhysicalRel visit(
+            final StreamPhysicalRel rel, final ImmutableBitSet requireDeterminism) {
+        if (rel instanceof StreamPhysicalSink) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // for update streaming, when
+                // 1. sink with pk:
+                // upsert sink, update by pk, ideally pk == input.upsertKey,
+                // (otherwise upsertMaterialize will handle it)
+
+                // 1.1 input.upsertKey nonEmpty -> not care about NDU
+                // 1.2 input.upsertKey isEmpty -> retract by complete row, must not contain NDU
+
+                // once sink's requirement on pk was satisfied, no further request will be transited
+                // only when new requirement generated at stateful node which input has update
+                // (e.g., grouping keys)
+
+                // 2. sink without pk:
+                // retract sink, retract by complete row (all input columns should be deterministic)
+                // whether input.upsertKey is empty or not, must not contain NDU
+                StreamPhysicalSink sink = (StreamPhysicalSink) rel;
+                int[] primaryKey =
+                        sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes();
+                ImmutableBitSet requireInputDeterminism;
+                if (sink.upsertMaterialize() || null == primaryKey || primaryKey.length == 0) {
+                    // SinkUpsertMaterializer only support no upsertKey mode, it says all input
+                    // columns should be deterministic (same as no primary key defined on sink)
+                    // TODO should optimize it after SinkUpsertMaterializer support upsertKey
+                    // FLINK-28569.
+                    requireInputDeterminism =
+                            ImmutableBitSet.range(sink.getInput().getRowType().getFieldCount());
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.of(primaryKey);
+                }
+                return transmitDeterminismRequirement(sink, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalLegacySink<?>) {
+            if (inputInsertOnly(rel)) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                StreamPhysicalLegacySink sink = (StreamPhysicalLegacySink) rel;
+                TableSchema tableSchema = sink.sink().getTableSchema();
+                Optional<UniqueConstraint> primaryKey = tableSchema.getPrimaryKey();
+                List<String> columns = Arrays.asList(tableSchema.getFieldNames());
+                // SinkUpsertMaterializer does not support legacy sink
+                ImmutableBitSet requireInputDeterminism;
+                if (primaryKey.isPresent()) {
+                    requireInputDeterminism =
+                            ImmutableBitSet.of(
+                                    primaryKey.get().getColumns().stream()
+                                            .map(col -> columns.indexOf(col))
+                                            .collect(Collectors.toList()));
+                } else {
+                    requireInputDeterminism = ImmutableBitSet.range(columns.size());
+                }
+                return transmitDeterminismRequirement(rel, requireInputDeterminism);
+            }
+        } else if (rel instanceof StreamPhysicalCalcBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                // for append stream, not care about NDU
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // if input has updates, any non-deterministic conditions are not acceptable, also
+                // requireDeterminism should be satisfied.
+                StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel;
+                checkNonDeterministicRexProgram(requireDeterminism, calc.getProgram(), calc);
+
+                // evaluate required determinism from input
+                List<RexNode> projects =
+                        calc.getProgram().getProjectList().stream()
+                                .map(expr -> calc.getProgram().expandLocalRef(expr))
+                                .collect(Collectors.toList());
+                Map<Integer, Integer> outFromSourcePos = extractSourceMapping(projects);
+                List<Integer> conv2Inputs =
+                        requireDeterminism.toList().stream()
+                                .map(
+                                        out ->
+                                                Optional.ofNullable(outFromSourcePos.get(out))
+                                                        .orElseThrow(
+                                                                () ->
+                                                                        new TableException(
+                                                                                String.format(
+                                                                                        "Invalid pos:%d over projection:%s",
+                                                                                        out,
+                                                                                        calc
+                                                                                                .getProgram()))))
+                                .filter(index -> index != -1)
+                                .collect(Collectors.toList());
+
+                return transmitDeterminismRequirement(calc, ImmutableBitSet.of(conv2Inputs));
+            }
+        } else if (rel instanceof StreamPhysicalCorrelateBase) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                // check if non-deterministic condition (may exist after FLINK-7865 was fixed).
+                StreamPhysicalCorrelateBase correlate = (StreamPhysicalCorrelateBase) rel;
+                if (correlate.condition().isDefined()) {
+                    RexNode rexNode = correlate.condition().get();
+                    checkNonDeterministicCondition(rexNode, correlate);
+                }
+                // check if it is a non-deterministic function
+                int leftFieldCnt = correlate.inputRel().getRowType().getFieldCount();
+                Optional<String> ndCall = getNonDeterministicCallName(correlate.scan().getCall());
+                if (ndCall.isPresent()) {
+                    // all columns from table function scan cannot satisfy the required determinism
+                    List<Integer> unsatisfiedColumns =
+                            requireDeterminism.toList().stream()
+                                    .filter(index -> index >= leftFieldCnt)
+                                    .collect(Collectors.toList());
+                    if (!unsatisfiedColumns.isEmpty()) {
+                        String errorMsg =
+                                generateNonDeterministicColumnsErrorMessage(
+                                        unsatisfiedColumns,
+                                        correlate.getRowType(),
+                                        correlate,
+                                        null,
+                                        ndCall);
+                        throw new TableException(errorMsg);
+                    }
+                }
+                // evaluate required determinism from input
+                List<Integer> fromLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+                if (fromLeft.isEmpty()) {
+                    return transmitDeterminismRequirement(correlate, NO_REQUIRED_DETERMINISM);
+                }
+                return transmitDeterminismRequirement(correlate, ImmutableBitSet.of(fromLeft));
+            }
+
+        } else if (rel instanceof StreamPhysicalLookupJoin) {
+            if (inputInsertOnly(rel) || requireDeterminism.isEmpty()) {
+                return transmitDeterminismRequirement(rel, NO_REQUIRED_DETERMINISM);
+            } else {
+                /**
+                 * if input has updates, the lookup join may produce non-deterministic result itself
+                 * due to backed lookup source which data may change over time, we can try to
+                 * eliminate this non-determinism by adding materialization to the join operator,
+                 * but still exists non-determinism we cannot solve: 1. join condition 2. the inner
+                 * calc in lookJoin.
+                 */
+                StreamPhysicalLookupJoin lookupJoin = (StreamPhysicalLookupJoin) rel;
+
+                // required determinism cannot be satisfied even upsert materialize was enabled if:
+                // 1. remaining join condition contains non-deterministic call
+                if (lookupJoin.remainingCondition().isDefined()) {
+                    RexNode rexNode = lookupJoin.remainingCondition().get();
+                    checkNonDeterministicCondition(rexNode, lookupJoin);
+                }
+                // 2. inner calc in lookJoin contains either non-deterministic condition or calls
+                if (lookupJoin.calcOnTemporalTable().isDefined()) {
+                    checkNonDeterministicRexProgram(
+                            requireDeterminism, lookupJoin.calcOnTemporalTable().get(), lookupJoin);
+                }
+
+                // Try to resolve non-determinism by adding materialization which can eliminate
+                // non-determinism produced by lookup join via an evolving source.
+                int leftFieldCnt = lookupJoin.getInput().getRowType().getFieldCount();
+                List<Integer> requireRight =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index >= leftFieldCnt)
+                                .collect(Collectors.toList());
+                boolean omitUpsertMaterialize = false;
+                // two optimizations: 1. no fields from lookup source was required 2. lookup key
+                // contains pk and no requirement on other fields we can omit materialization,
+                // otherwise upsert materialize can not be omitted.
+                if (requireRight.isEmpty()) {
+                    omitUpsertMaterialize = true;
+                } else {
+                    int[] outputPkIdx = lookupJoin.getOutputPrimaryKeyIndexes();
+                    ImmutableBitSet outputPkBitSet = ImmutableBitSet.of(outputPkIdx);
+                    // outputPkIdx need to used so not using #lookupKeyContainsPrimaryKey directly.
+                    omitUpsertMaterialize =
+                            null != outputPkIdx
+                                    && outputPkIdx.length > 0
+                                    && Arrays.stream(outputPkIdx)
+                                            .allMatch(
+                                                    index ->
+                                                            lookupJoin
+                                                                    .allLookupKeys()
+                                                                    .contains(index))
+                                    && requireRight.stream()
+                                            .allMatch(index -> outputPkBitSet.get(index));
+                }
+                List<Integer> requireLeft =
+                        requireDeterminism.toList().stream()
+                                .filter(index -> index < leftFieldCnt)
+                                .collect(Collectors.toList());
+
+                if (omitUpsertMaterialize) {
+                    return transmitDeterminismRequirement(
+                            lookupJoin, ImmutableBitSet.of(requireLeft));
+                } else {
+                    // enable materialize for lookup join
+                    return transmitDeterminismRequirement(
+                            lookupJoin.copy(true), ImmutableBitSet.of(requireLeft));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalTableSourceScan) {
+            // tableScan has no input, so only check meta data from cdc source
+            if (!requireDeterminism.isEmpty()) {
+                StreamPhysicalTableSourceScan tableScan = (StreamPhysicalTableSourceScan) rel;
+                boolean insertOnly =
+                        tableScan.tableSource().getChangelogMode().containsOnly(RowKind.INSERT);
+                boolean supportsReadingMetadata =
+                        tableScan.tableSource() instanceof SupportsReadingMetadata;
+                if (!insertOnly && supportsReadingMetadata) {
+                    TableSourceTable sourceTable =
+                            tableScan.getTable().unwrap(TableSourceTable.class);
+                    // check if requireDeterminism contains metadata column
+                    List<Column.MetadataColumn> metadataColumns =
+                            DynamicSourceUtils.extractMetadataColumns(
+                                    sourceTable.contextResolvedTable().getResolvedSchema());
+                    Set<String> metaColumnSet =
+                            metadataColumns.stream()
+                                    .map(col -> col.getName())
+                                    .collect(Collectors.toSet());
+                    List<String> columns = tableScan.getRowType().getFieldNames();
+                    List<String> metadataCauseErr = new ArrayList<>();
+                    for (int index = 0; index < columns.size(); index++) {
+                        String column = columns.get(index);
+                        if (metaColumnSet.contains(column) && requireDeterminism.get(index)) {
+                            metadataCauseErr.add(column);
+                        }
+                    }
+                    if (!metadataCauseErr.isEmpty()) {
+                        StringBuilder errorMsg = new StringBuilder();
+                        errorMsg.append("The metadata column(s): '")
+                                .append(String.join(", ", metadataCauseErr.toArray(new String[0])))
+                                .append("' in cdc source may cause wrong result or error on")
+                                .append(" downstream operators, please consider removing these")
+                                .append(" columns or use a non-cdc source that only has insert")
+                                .append(" messages.\nsource node:\n")
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                tableScan,
+                                                SqlExplainLevel.DIGEST_ATTRIBUTES,
+                                                false,
+                                                true,
+                                                false,
+                                                true));
+                        throw new TableException(errorMsg.toString());
+                    }
+                }
+            }
+            return rel;
+        } else if (rel instanceof StreamPhysicalLegacyTableSourceScan
+                || rel instanceof StreamPhysicalDataStreamScan
+                || rel instanceof StreamPhysicalValues) {
+            // not cdc source, end visit
+            return rel;
+        } else if (rel instanceof StreamPhysicalGroupAggregateBase) {
+            // output row type = grouping keys + aggCalls
+            StreamPhysicalGroupAggregateBase groupAgg = (StreamPhysicalGroupAggregateBase) rel;
+            if (inputInsertOnly(groupAgg)) {
+                // no further requirement to input, only check if can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            groupAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(groupAgg.aggCalls()),
+                            groupAgg.getRowType(),
+                            groupAgg);
+                }
+                return transmitDeterminismRequirement(groupAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        groupAgg,
+                        ImmutableBitSet.range(groupAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowAggregateBase) {
+            // output row type = grouping keys + aggCalls + windowProperties
+            // same logic with 'groupAgg' but they have no common parent
+            StreamPhysicalWindowAggregateBase windowAgg = (StreamPhysicalWindowAggregateBase) rel;
+            if (inputInsertOnly(windowAgg)) {
+                // no further requirement to input, only check if it can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    checkUnsatisfiedDeterminism(
+                            requireDeterminism,
+                            windowAgg.grouping().length,
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(windowAgg.aggCalls()),
+                            windowAgg.getRowType(),
+                            windowAgg);
+                }
+                return transmitDeterminismRequirement(windowAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // agg works under retract mode if input is not insert only, and requires all input
+                // columns be deterministic
+                return transmitDeterminismRequirement(
+                        windowAgg,
+                        ImmutableBitSet.range(windowAgg.getInput().getRowType().getFieldCount()));
+            }
+        } else if (rel instanceof StreamPhysicalExpand) {
+            // Expand is an internal operator only for plan rewriting currently, so only remove the
+            // expandIdIndex from requireDeterminism. We also skip checking if input has updates due
+            // to this is a non-stateful node which never changes the changelog mode.
+            StreamPhysicalExpand expand = (StreamPhysicalExpand) rel;
+            return transmitDeterminismRequirement(
+                    expand, requireDeterminism.except(ImmutableBitSet.of(expand.expandIdIndex())));
+        } else if (rel instanceof CommonPhysicalJoin) {
+            // output row type = left row type + right row type
+            CommonPhysicalJoin join = (CommonPhysicalJoin) rel;
+            StreamPhysicalRel leftRel = (StreamPhysicalRel) join.getLeft();
+            StreamPhysicalRel rightRel = (StreamPhysicalRel) join.getRight();
+            boolean leftInputHasUpdate = !inputInsertOnly(leftRel);
+            boolean rightInputHasUpdate = !inputInsertOnly(rightRel);
+            boolean innerOrSemi =
+                    join.joinSpec().getJoinType() == FlinkJoinType.INNER
+                            || join.joinSpec().getJoinType() == FlinkJoinType.SEMI;
+            /**
+             * we do not distinguish the time attribute condition in interval/temporal join from
+             * regular/window join here because: rowtime field always from source, proctime is not
+             * limited (from source), when proctime appended to an update row without upsertKey then
+             * result may goes wrong, in such a case proctime( was materialized as
+             * PROCTIME_MATERIALIZE(PROCTIME())) is equal to a normal dynamic temporal function and
+             * will be validated in calc node.
+             */
+            Optional<String> ndCall = getNonDeterministicCallName(join.getCondition());
+            if ((leftInputHasUpdate || rightInputHasUpdate || !innerOrSemi) && ndCall.isPresent()) {
+                // when output has update, the join condition cannot be non-deterministic:
+                // 1. input has update -> output has update
+                // 2. input insert only and is not innerOrSemi join -> output has update
+                throw new TableException(
+                        generateNonDeterministicConditionErrorMessage(
+                                ndCall.get(), join.getCondition(), (StreamPhysicalRel) join));
+            }
+            int leftFieldCnt = leftRel.getRowType().getFieldCount();
+            StreamPhysicalRel newLeft =
+                    visitJoinChild(
+                            requireDeterminism,
+                            leftRel,
+                            leftInputHasUpdate,
+                            leftFieldCnt,
+                            true,
+                            join.joinSpec().getLeftKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(leftRel, join.joinSpec().getLeftKeys())));
+            StreamPhysicalRel newRight =
+                    visitJoinChild(
+                            requireDeterminism,
+                            rightRel,
+                            rightInputHasUpdate,
+                            leftFieldCnt,
+                            false,
+                            join.joinSpec().getRightKeys(),
+                            // TODO remove this conversion when scala-free was total done.
+                            scala.collection.JavaConverters.seqAsJavaList(
+                                    join.getUniqueKeys(rightRel, join.joinSpec().getRightKeys())));
+
+            return (StreamPhysicalRel)
+                    join.copy(
+                            join.getTraitSet(),
+                            join.getCondition(),
+                            newLeft,
+                            newRight,
+                            join.getJoinType(),
+                            join.isSemiJoin());
+
+        } else if (rel instanceof StreamPhysicalOverAggregateBase) {
+            // output row type = input row type + overAgg outputs
+            StreamPhysicalOverAggregateBase overAgg = ((StreamPhysicalOverAggregateBase) rel);
+            if (inputInsertOnly(overAgg)) {
+                // no further requirement to input, only check if the agg outputs can satisfy the
+                // requiredDeterminism
+                if (!requireDeterminism.isEmpty()) {
+                    int inputFieldCnt = overAgg.getInput().getRowType().getFieldCount();
+                    OverSpec overSpec = OverAggregateUtil.createOverSpec(overAgg.logicWindow());
+                    // add aggCall's input
+                    int aggOutputIndex = inputFieldCnt;
+                    for (OverSpec.GroupSpec groupSpec : overSpec.getGroups()) {
+                        checkUnsatisfiedDeterminism(
+                                requireDeterminism,
+                                aggOutputIndex,
+                                groupSpec.getAggCalls(),
+                                overAgg.getRowType(),
+                                overAgg);
+                        aggOutputIndex += groupSpec.getAggCalls().size();
+                    }
+                }
+                return transmitDeterminismRequirement(overAgg, NO_REQUIRED_DETERMINISM);
+            } else {
+                // OverAgg does not support input with updates currently, so this branch will not be
+                // reached for now.
+
+                // We should append partition keys and order key to requireDeterminism
+                return transmitDeterminismRequirement(
+                        overAgg, mappingRequireDeterminismToInput(requireDeterminism, overAgg));
+            }
+        } else if (rel instanceof StreamPhysicalRank) {
+            // if outputRankNumber:  output row type = input row type + rank number type
+            // else keeps the same as input
+            StreamPhysicalRank rank = (StreamPhysicalRank) rel;
+            if (inputInsertOnly(rank)) {
+                // rank output is deterministic when input is insert only, so required determinism
+                // always be satisfied here.
+                return transmitDeterminismRequirement(rank, NO_REQUIRED_DETERMINISM);
+            } else {
+                int inputFieldCnt = rank.getInput().getRowType().getFieldCount();
+                if (rank.rankStrategy() instanceof RankProcessStrategy.UpdateFastStrategy) {
+                    // in update fast mode, pass required determinism excludes partition keys and
+                    // order key
+                    ImmutableBitSet.Builder bitSetBuilder = ImmutableBitSet.builder();
+                    rank.partitionKey().toList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    rank.orderKey().getKeys().toIntegerList().stream()
+                            .forEach(index -> bitSetBuilder.set(index));
+                    if (rank.outputRankNumber()) {
+                        // exclude last column
+                        bitSetBuilder.set(inputFieldCnt);
+                    }
+                    return transmitDeterminismRequirement(
+                            rank, requireDeterminism.except(bitSetBuilder.build()));
+                } else if (rank.rankStrategy() instanceof RankProcessStrategy.RetractStrategy) {
+                    // in retract mode then require all input columns be deterministic
+                    return transmitDeterminismRequirement(
+                            rank, ImmutableBitSet.range(inputFieldCnt));
+                } else {
+                    // AppendFastStrategy only applicable for insert only input, so the undefined
+                    // strategy is not as expected here.
+                    throw new TableException(
+                            String.format(
+                                    "Can not infer the determinism for unsupported rank strategy: %s, this is a bug, please file an issue.",
+                                    rank.rankStrategy()));
+                }
+            }
+        } else if (rel instanceof StreamPhysicalDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalDeduplicate dedup = (StreamPhysicalDeduplicate) rel;
+            if (inputInsertOnly(dedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(dedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // Deduplicate always has unique key currently(exec node has null check and inner
+                // state only support data with keys), so only pass the left columns of required
+                // determinism to input.
+                return transmitDeterminismRequirement(
+                        dedup,
+                        requireDeterminism.except(ImmutableBitSet.of(dedup.getUniqueKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowDeduplicate) {
+            // output row type same as input and does not change output columns' order
+            StreamPhysicalWindowDeduplicate winDedup = (StreamPhysicalWindowDeduplicate) rel;
+            if (inputInsertOnly(winDedup)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winDedup, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowDeduplicate does not support input with updates currently, so this branch
+                // will not be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                return transmitDeterminismRequirement(
+                        winDedup,
+                        requireDeterminism
+                                .clear(winDedup.orderKey())
+                                .union(ImmutableBitSet.of(winDedup.partitionKeys())));
+            }
+        } else if (rel instanceof StreamPhysicalWindowRank) {
+            StreamPhysicalWindowRank winRank = (StreamPhysicalWindowRank) rel;
+            if (inputInsertOnly(winRank)) {
+                // similar to rank, output is deterministic when input is insert only, so required
+                // determinism always be satisfied here.
+                return transmitDeterminismRequirement(winRank, NO_REQUIRED_DETERMINISM);
+            } else {
+                // WindowRank does not support input with updates currently, so this branch will not
+                // be reached for now.
+
+                // only append partition keys, no need to process order key because it always comes
+                // from window
+                int inputFieldCnt = winRank.getInput().getRowType().getFieldCount();
+                return transmitDeterminismRequirement(
+                        winRank,
+                        requireDeterminism
+                                .intersect(ImmutableBitSet.range(inputFieldCnt))
+                                .union(winRank.partitionKey()));
+            }
+        } else if (rel instanceof StreamPhysicalWindowTableFunction) {
+            // output row type = input row type + window attributes
+            StreamPhysicalWindowTableFunction winTVF = (StreamPhysicalWindowTableFunction) rel;
+            if (inputInsertOnly(winTVF)) {
+                return transmitDeterminismRequirement(winTVF, NO_REQUIRED_DETERMINISM);
+            } else {
+                // pass the left columns of required determinism to input exclude window attributes
+                return transmitDeterminismRequirement(
+                        winTVF,
+                        requireDeterminism.intersect(
+                                ImmutableBitSet.range(
+                                        winTVF.getInput().getRowType().getFieldCount())));
+            }
+        } else if (rel instanceof StreamPhysicalChangelogNormalize

Review Comment:
   StreamPhysicalChangelogNormalize is a special op here, it is generated by planner for  upsert source only, it must have upsert keys that equal to the source's pk, so here can just simply do transmit.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1205950009

   rebase master to resolve failed e2e case (4e790646c Revert [FLINK-28373])


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1205354370

   @flinkbot run azure


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on PR #20393:
URL: https://github.com/apache/flink/pull/20393#issuecomment-1201908318

   @godfreyhe thanks for reviewing this!  I'm rewriting the new scala class to java version(the progress is not as fast as expected), and will split the physical operator implementation into [FLINK-28568](https://issues.apache.org/jira/browse/FLINK-28568)


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lincoln-lil commented on a diff in pull request #20393: [FLINK-28570][table-planner] Introduces a StreamNonDeterministicPlanResolver to validate and try to solve (lookup join only) the non-deterministic updates problem which may cause wrong result or error

Posted by GitBox <gi...@apache.org>.
lincoln-lil commented on code in PR #20393:
URL: https://github.com/apache/flink/pull/20393#discussion_r935658039


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala:
##########
@@ -62,14 +62,19 @@ class StreamPhysicalIncrementalGroupAggregate(
     inputRel: RelNode,
     val partialAggGrouping: Array[Int],
     val partialAggCalls: Array[AggregateCall],
-    val finalAggGrouping: Array[Int],

Review Comment:
   Do you mean the attribute name in `StreamPhysicalGroupAggregateBase`?
   I was thinking the partial-final is the internal concept for two-phase aggregate, so tend to omit the 'final' prefix in the base class which I think can both represents
   grouping in one-phase agg and distinguishes from partialGroupings in two-phase agg.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org