JingsongLi commented on a change in pull request #16630:
URL: https://github.com/apache/flink/pull/16630#discussion_r680816074



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
##########
@@ -89,6 +91,17 @@
                                     + "but there will be additional overhead."
                                     + "Default is true.");
 
+    @Experimental
+    public static final ConfigOption<Boolean>
+            TABLE_EXEC_DEDUPLICATE_MINIBATCH_ALL_CHANGELOG_ENABLED =
+                    
ConfigOptions.key("table.exec.deduplicate.mini-batch.all.change-log.enabled")
+                            .booleanType()
+                            .defaultValue(true)
+                            .withDescription(
+                                    "Set whether send all change log for 
row-time mini-batch."
+                                            + "If false, Flink will only send 
the latest change log to downstream like proc-time."

Review comment:
       If downstream is: 
https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/table/concepts/versioned_tables/#versioned-table-views
   This optimization can not be enabled.
   
   We should add description here.

##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/RowTimeMiniBatchLatestChangelogDeduplicateFunction.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.operators.deduplicate;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.checkInsertOnly;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.isDuplicate;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.updateDeduplicateResult;
+
+/**
+ * This function is used to get the first or last row for every key partition 
in miniBatch mode. But
+ * only send latest change log to downstream.
+ */
+public class RowTimeMiniBatchLatestChangelogDeduplicateFunction
+        extends MiniBatchDeduplicateFunctionBase<RowData, RowData, RowData, 
RowData, RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final TypeSerializer<RowData> serializer;
+    private final boolean generateUpdateBefore;
+    private final boolean generateInsert;
+    private final int rowtimeIndex;
+    private final boolean keepLastRow;
+
+    public RowTimeMiniBatchLatestChangelogDeduplicateFunction(
+            InternalTypeInfo<RowData> typeInfo,
+            TypeSerializer<RowData> serializer,
+            long minRetentionTime,
+            int rowtimeIndex,
+            boolean generateUpdateBefore,
+            boolean generateInsert,
+            boolean keepLastRow) {
+        super(typeInfo, minRetentionTime);
+        this.serializer = serializer;
+        this.generateUpdateBefore = generateUpdateBefore;
+        this.generateInsert = generateInsert;
+        this.rowtimeIndex = rowtimeIndex;
+        this.keepLastRow = keepLastRow;
+    }
+
+    @Override
+    public RowData addInput(@Nullable RowData value, RowData input) throws 
Exception {
+        if (isDuplicate(value, input, rowtimeIndex, keepLastRow)) {
+            return serializer.copy(input);
+        }
+        return value;
+    }
+
+    @Override
+    public void finishBundle(Map<RowData, RowData> buffer, Collector<RowData> 
out)
+            throws Exception {
+        for (Map.Entry<RowData, RowData> entry : buffer.entrySet()) {
+            RowData currentKey = entry.getKey();
+            RowData bufferedRow = entry.getValue();
+            ctx.setCurrentKey(currentKey);
+            RowData preRow = state.value();
+            // Note: we output all changelog here rather than comparing the 
first and the last

Review comment:
       wrong comments here...
   We just broke this constraint.

##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/RowTimeMiniBatchLatestChangelogDeduplicateFunction.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.operators.deduplicate;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.checkInsertOnly;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.isDuplicate;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.updateDeduplicateResult;
+
+/**
+ * This function is used to get the first or last row for every key partition 
in miniBatch mode. But
+ * only send latest change log to downstream.
+ */
+public class RowTimeMiniBatchLatestChangelogDeduplicateFunction

Review comment:
       You need add a Harness test for this class, a example is 
`ProcTimeDeduplicateKeepFirstRowFunctionTest`.

##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/RowTimeMiniBatchLatestChangelogDeduplicateFunction.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.operators.deduplicate;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.checkInsertOnly;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.isDuplicate;
+import static 
org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.updateDeduplicateResult;
+
+/**
+ * This function is used to get the first or last row for every key partition 
in miniBatch mode. But
+ * only send latest change log to downstream.
+ */
+public class RowTimeMiniBatchLatestChangelogDeduplicateFunction

Review comment:
       `RowTimeMiniBatchLatestChangeDeduplicateFunction`?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
##########
@@ -89,6 +91,17 @@
                                     + "but there will be additional overhead."
                                     + "Default is true.");
 
+    @Experimental
+    public static final ConfigOption<Boolean>
+            TABLE_EXEC_DEDUPLICATE_MINIBATCH_ALL_CHANGELOG_ENABLED =
+                    
ConfigOptions.key("table.exec.deduplicate.mini-batch.all.change-log.enabled")

Review comment:
       We can make a reverse name.
   For example: `table.exec.deduplicate.mini-batch.compact-changes`, default 
value is false.




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

To unsubscribe, e-mail: [email protected]

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


Reply via email to