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



##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializer.java
##########
@@ -96,36 +98,39 @@ public void open() throws Exception {
 
     @Override
     public void processElement(StreamRecord<RowData> element) throws Exception 
{
-        RowData row = element.getValue();
-        boolean isInsertOp = row.getRowKind() == INSERT || row.getRowKind() == 
UPDATE_AFTER;
-        // Always set the RowKind to INSERT, so that we can compare rows 
correctly (RowKind will
-        // be ignored)
-        row.setRowKind(INSERT);
+        final RowData row = element.getValue();
         List<RowData> values = state.value();
         if (values == null) {
             values = new ArrayList<>(2);
         }
 
-        if (isInsertOp) {
-            values.add(row);
-            // Update to this new one
-            collector.collect(row);
-        } else {
-            int lastIndex = values.size() - 1;
-            int index = removeFirst(values, row);
-            if (index == -1) {
-                LOG.info(STATE_CLEARED_WARN_MSG);
-                return;
-            }
-            if (values.isEmpty()) {
-                // Delete this row
-                row.setRowKind(DELETE);
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                row.setRowKind(values.isEmpty() ? INSERT : UPDATE_AFTER);
+                values.add(row);

Review comment:
       Can we still keep update an insert row? This will make understanding 
logic easier. So we don't need modify equals in `removeFirst`.




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