You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/07/30 18:20:04 UTC

[GitHub] [iceberg] Reo-LEI opened a new pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Reo-LEI opened a new pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898


   For CDC data, usually we will simply synchronize the data to iceberg.  And we can use FlinkSQL to do this.
   
   Consider we use FlinkSQL define a sourc of cdc-connector and a sink of iceberg-connector, and simply INSERT INTO cdc data from src to snk. Finally we will get a job graph like this:
   ```
   FlinkCDCSource ---forward---> Filter ---forward---> IcebergStreamWriter ---forward---> IcebergFilesCommitter
   ```
   That will be work fine, because all opr parallelism is 1, all cdc data will be distributed to one `IcebergStreamWriter`.
   
   But once we setting the default parallelism of job, example 3 parrallelism, we will get another job graph as follow:
   ```
                                 +---> Filter ---forward---> IcebergStreamWriter ---+
   FlinkCDCSource ---rebalance---+---> Filter ---forward---> IcebergStreamWriter ---+---rebalance---> IcebergFilesCommitter
                                 +---> Filter ---forward---> IcebergStreamWriter ---+
   ```
   Now the CDC data will be distributed to three different `IcebergStreamWriter`, because we have not keyBy the primary key. If we insert one row and update it as follow, we will get an duplicate row.
   ```
   // first field is id(primary key), second field is value
   INSERT <1, aaa>
   UPDATE <1, bbb>
   ```
   Because the change log wil be rebalance to different `IcebergStreamWriter`, first writer will get +I record, second writer will get -U record and last writer will get +U record. Due to second writer's `insertedRowMap` is empty and the pos-delete(-U) will be ignore. Finally we got an duplicate row.
   
   I try to fix that by add keyBy before `IcebergStreamWriter` and chaining the `Source` and `Filter` opr if use FlinkSQL.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] liubo1022126 commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
liubo1022126 commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1043817661


   @Reo-LEI @rdblue hi,boys,We are currently using the flink upsert function. In which version is this PR planned to be released?


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink 1.14: Add EqualityFieldKeySelector.

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815561381



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer in order.
+ */
+class EqualityFieldKeySelector implements KeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final RowType flinkSchema;
+  private final Schema deleteSchema;
+
+  private transient RowDataWrapper rowDataWrapper;
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {

Review comment:
       Done, and thanks for reviewing!




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-919277576


   > @Reo-LEI Thanks for the contribution, sounds like I'm late for the reviewing. Would you like to resolve the conflicts , I think I will get this reviewing tomorrow, thanks.
   
   @openinx Thanks for your reply. Since I discuss this PR with @kbendick  @stevenzwu, I think the implementation of setting `Filter` opr parallelism and use `CombinedKeySelector` to key by equality key and partition key maybe not a good way to reslove this problem. I'm plan to refactor this, and maybe you could review this PR after that. :)


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] coolderli commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
coolderli commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891486040


   
   > Actually set `write.distribution-mode = hash` can not resolve this. Because CDC data shoule be distributed to primary key but not partition fields. For example, an iceberg table has equalityFields (dt, hour, id) and partition fields (dt, hour). If set `write.distribution-mode = hash`, the CDC data will be keyBy (dt, hour), and we can not ensure same id CDC data will send to same `IcebergStreamWriter`.
   
   I'm a litter confused. Assume that the upstream data is orderly, if the CDC data are key by (dt, hour), the same id should be send to the same `IcebergStreamWriter`.
   
   `dt=2021-08-03, hour=10, id=1` and `dt=2021-08-03, hour=10, id=2` should be the same `IcebergStreamWriter`. I think a partition is just a subset of the primary key. If anything is wrong, please let me know. Thanks!
   
   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r683761791



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}

Review comment:
       Ah my bad. My eyes deceive me =) 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-903094697


   > @Reo-LEI I left a comment in the issue #2918. Let's continue the problem discussion there.
   > 
   > You probably want to rebase the PR properly. right now, it shows 1,000+ files modified.
   
   My mistake, I have been fix that. @stevenzwu 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891130393


   Thanks for your review and give me a lot of advice @kbendick . I open an issue #2918 to illustrate this problem more detailly, I hope that can help you to understand this.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697882281



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -369,6 +373,27 @@ private String operatorName(String suffix) {
         default:
           throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode);
       }
+
+      if (keySelector != null) {
+        return input.keyBy(keySelector);
+      }
+      return input;
+    }
+
+    private KeySelector<RowData, String> getKeySelector(List<Integer> equalityFieldIds, PartitionSpec partitionSpec,
+        Schema schema, RowType rowType) {
+      boolean hasPrimaryKey = equalityFieldIds != null && !equalityFieldIds.isEmpty();
+      boolean hasPartitionKey = partitionSpec != null && !partitionSpec.isUnpartitioned();
+
+      if (hasPrimaryKey && hasPartitionKey) {
+        return new CombinedKeySelector(partitionSpec, equalityFieldIds, schema, rowType);

Review comment:
        As far as I know, between equality key and partition key are two independent fields, there is no restriction between them. So I add `CombinedKeySelector` to maintain the independence between them. But that will defeats the purpose of hash distribution as you say. 
   
   restrict enable hash distribution mode when `equalityFieldIds` is not null is one way to reslove this problem, but I think should we restrict partition key must be a subset of equality key to reslove this? Actualy, I have little confuse between equality key and partition key. 
   
   I think partition key should be a subset of equality key, because when user define the equality key in partitioned table that is mean one row should be unique in each partition. Such as we got a table partitioned by <day, hour>, we should ensure equality key contain partition key like <day, hour, id>, but not like <id>. I think the later is confusing and make no sense.
   Just like a MySQL table which primary key is <userId>, and sharding data by <day, hour> to different table, and still require keep <userId> is unique in global.
   
   Once we can ensure partition key is a subset of equality key, we can remove the `CombinedKeySelector` and use equality key sheffle data.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r779280068



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StringBuilder builder = new StringBuilder("Key(");

Review comment:
       nit: do we need the `key(...)` wrap in the string?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719900717



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;
+    wrapper = wrapper == null ? StructLikeWrapper.forType(deleteSchema.asStruct()) : wrapper;
+    return wrapper.set(projection.wrap(lazyRowDataWrapper().wrap(row)));

Review comment:
       got it. thx for explaining. there are so many wrap/wrapper in this line of code. maybe rename `wrapper` to `structLikeWrapper`?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718136792



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.

Review comment:
       NIt: `it's` -> `its` ?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693619492



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -140,13 +141,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case), So we need to set the map opr
+        // parallelism as it's input to keep map opr chain to input, and ensure input stream will not be rebalanced.

Review comment:
       I agree with this change. Since map operator is part of the Flink sink, it is ok to set the parallelism.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681841099



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       The `Filter` opr is appended by flink to prevenet send null data to downstream. So the `Fliter` opr will alway exists when we use FlinkSQL to construct our job, and I think the opr name can not be change by user(so far I'm not found relevant api to rename an opr by sql).
   
   And I think we can simply set the `Filter` opr parallelism as its input and keep `Filter` chain its input opr in all case and without worrying some side-effect. Because `Filter` will not effect the input stream.
   
   Here we only change the forward opr parallelism when forward opr is `Filter`, I think that is generally enough and safe condition to keep `Filter` chain its input




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891138179


   > Additionally, I am wondering if there is possibly a need for different key selectors in the case of `range` partitioning. If one has an UPSERT stream of adds and deletes in general, will they be affected by this issue as well?
   I think we don't need a key selector for `range` partitioning, because iceberg-flink don't support `range` partitioning yet. And I think `range` partitioning is not easy to impl in streaming system, and we can't ignore this case now.
   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-930761242


   Thanks to @openinx and @stevenzwu for review! I have been refactor the `EqualityFieldKeySelector` by `StructLikeWrapper` and resolve some comment. Maybe you can take another look for this. :)


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r715845154



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }

Review comment:
       For line 420 to 429 is trying to resolve the duplicate rows problem. If equality fields are already specified, in default, that will shuffle records by `equalityFieldIds` for unpartitioned and partitioned table. 
   
   The all cases about `Distribution Mode` discussion as follow: 
   
   Case | Stremaing Type | Distribution Mode | Primary Key | Partition Key | Shuffle by
   -- | -- | -- | -- | -- | --
   1 | retract | NONE | userId |   | userId
   2 | retract | NONE | userId | day, hour | userId
   3 | retract | HASH | userId |   | userId
   4 | retract | HASH | userId | day, hour | day, hour
   5 | upsert | NONE | userId |   | userId
   6 | upsert | NONE | day, hour, userId | day, hour | day, hour, userId
   7 | upsert | HASH | userId |   | userId
   8 | upsert | HASH | day, hour, userId | day, hour | day, hour
   
   
   According to the solution of https://github.com/apache/iceberg/pull/2898#issuecomment-926830884, we will get the job graph as follow for all cases.
   ```
   ChangelogStream-1 ---forward---> Filter/Map-1 ---+      +---> IcebergStreamWriter-1 ---+
             ...                          ...       +-hash-+             ...              +-rebalance---> IcebergFilesCommitter
   ChangelogStream-m ---forward---> Filter/Map-m ---+      +---> IcebergStreamWriter-n ---+
   ```
   Now, we can ensure all cdc/upsert record will send to correct `IcebergStreamWriter` and no -U message will be ignored.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-890355891


   > I think this may be avoided by setting `write.distribution-mode` to `hash`.
   > https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java#L322
   
   Actually set `write.distribution-mode = hash` can not resolve this. Because CDC data shoule be distributed to primary key but not partition fields. For example, an iceberg table has equalityFields (dt, hour, id) and partition fields (dt, hour). If set `write.distribution-mode = hash`, the CDC data will be keyBy (dt, hour), and we can not ensure same id CDC data will send to same `IcebergStreamWriter`. 
   
   And the second reason is , for FlinkSQL, `rowDataInput` contain a `FlinkCDCSource ` opr and a `Filter` opr. The `FlinkCDCSource ` opr parallelism is 1 and the `Filter` opr parallelism use job default parallelism. If we only keyBy `rowDataInput` output data to `IcebergStreamWriter`, we will get a job graph as follow: 
   ```
                                 +---> Filter ---+      +---> IcebergStreamWriter ---+
   FlinkCDCSource ---rebalance---+---> Filter ---+-hash-+---> IcebergStreamWriter ---+---rebalance---> IcebergFilesCommitter
                                 +---> Filter ---+      +---> IcebergStreamWriter ---+
   ```
   As result, the CDC data will be rebalance to `Filter`. That will make CDC data disorder(e.g. same `IcebergStreamWriter` will receive +U before -U), and we will get some strange result. So we need to keep the `Filter` have same parallelism to it's input opr.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-926830884


   After the above discussion, I think we can resolve the duplicate rows by add equality/partition shuffle between `inputStream` and `IcebergStreamWriter`. And for FlinkStreaming, we can resolve disorder problem by keep `map` operator parallelism same as `inputStream`. For FlinkSQL, we can resolve disorder problem by upgrade flink to 1.13 as I mentioned on https://github.com/apache/iceberg/issues/2918#issuecomment-926804817.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r716285008



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {
+    String[] values = new String[keySize];

Review comment:
       can we cache and reuse the array?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719051501



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",

Review comment:
       nit: do we need to include table schema in the error msg?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719051083



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;

Review comment:
       nit: maybe introduce `lazyStructProjection()` (similar to the ` lazyRowDataWrapper ` in base class)?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799692191



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();

Review comment:
       I think that Flink should default to the defined identifier field IDs for upsert behavior. If the user sets the columns explicitly, then we should check and warn if they don't match.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812615257



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());

Review comment:
       I deleted some unnecessary logs and rewritten the statements. Now the statements of the logs will not be repeated. Please take a look again.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780008147



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StringBuilder builder = new StringBuilder("Key(");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(lazyRowDataWrapper().wrap(row)));

Review comment:
       Good catch! I will move this out of the loop.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StringBuilder builder = new StringBuilder("Key(");

Review comment:
       I remove `Key(...)` format and use `[...]` to keep it same as `PartitionKey`.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -424,6 +435,9 @@ private String operatorName(String suffix) {
         case RANGE:
           LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",

Review comment:
       OK, this warn has been move to else block and I add different info log to mention different distribution mode will distribute rows by which fields.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r794165356



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.util.ByteBuffers;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, ByteBuffer> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: {}", schema.findField(equalityFieldIds.get(i)));
+
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public ByteBuffer getKey(RowData row) throws Exception {

Review comment:
       Agree!




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719111689



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",

Review comment:
       I think this is needed, that is helpful for user to find out the problem.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718139953



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       See: https://github.com/apache/iceberg/pull/2898/files#diff-a26cb8b3f0f5b0e6a9f110bb0b3fdd0ed625bfc9faf27e248f6446818cad4d69R436




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697887709



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},

Review comment:
       Because parallelism 2 could not cover this problem situation.  
   Suppose we have change log as follow:
   ```
   +I<1, "aaa">
   -U<1, "aaa">
   +U<1, "bbb"> 
   ```
   
   
   
   
   
   So I increase that to 4




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI edited a comment on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI edited a comment on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891138179


   > Additionally, I am wondering if there is possibly a need for different key selectors in the case of `range` partitioning. If one has an UPSERT stream of adds and deletes in general, will they be affected by this issue as well?
   
   I think we don't need a key selector for `range` partitioning, because iceberg-flink don't support `range` partitioning yet. And I think `range` partitioning is not easy to impl in streaming system, and we can't ignore this case now.
   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681116142



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -313,13 +320,21 @@ public Builder uidPrefix(String newPrefix) {
 
       switch (writeMode) {
         case NONE:
-          return input;
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, table.schema(), flinkRowType));
+          } else {
+            return input;
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
-          } else {
+          if (!partitionSpec.isUnpartitioned() && !equalityFieldIds.isEmpty()) {
+            return input.keyBy(new HybridKeySelector(partitionSpec, equalityFieldIds, iSchema, flinkRowType));
+          } else if (!partitionSpec.isUnpartitioned() && equalityFieldIds.isEmpty()) {
             return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+          } else if (partitionSpec.isUnpartitioned() && !equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, table.schema(), flinkRowType));
+          } else {
+            return input;

Review comment:
       That is make sense, I will add an utility method to do this.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697882281



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -369,6 +373,27 @@ private String operatorName(String suffix) {
         default:
           throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode);
       }
+
+      if (keySelector != null) {
+        return input.keyBy(keySelector);
+      }
+      return input;
+    }
+
+    private KeySelector<RowData, String> getKeySelector(List<Integer> equalityFieldIds, PartitionSpec partitionSpec,
+        Schema schema, RowType rowType) {
+      boolean hasPrimaryKey = equalityFieldIds != null && !equalityFieldIds.isEmpty();
+      boolean hasPartitionKey = partitionSpec != null && !partitionSpec.isUnpartitioned();
+
+      if (hasPrimaryKey && hasPartitionKey) {
+        return new CombinedKeySelector(partitionSpec, equalityFieldIds, schema, rowType);

Review comment:
        As far as I know, between equality key and partition key are two independent fields, there is no restriction between them. So I add `CombinedKeySelector` to maintain the independence between them. But that will defeats the purpose of hash distribution as you say. 
   
   Restrict enable hash distribution mode when `equalityFieldIds` is not null is one way to reslove this problem, but should we restrict partition key must be a subset of equality key to reslove this? 
   
   Actualy, I have little confuse between equality key and partition key. I think partition key should be a subset of equality key, because when user define the equality key in partitioned table that is mean one row should be unique in each partition. Such as we got a table partitioned by <day, hour>, we should ensure equality key contain partition key like <day, hour, id>, but not like <id>. I think the later is confusing and make no sense.
   Just like a MySQL table which primary key is <userId>, and sharding data by <day, hour> to different table, and still require keep <userId> is unique in global.
   
   Once we can ensure partition key is a subset of equality key, we can remove the `CombinedKeySelector` and use equality key sheffle data.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718285365



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;

Review comment:
       LGTM




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r724150784



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       I got your point, but I prefer to not expose `HASH-BY-EQUALITY-COLUMNS` to user. Because user has been setting the equality columns, we should make distribution mode as `HASH-BY-EQUALITY-COLUMNS` in default to ensure the correctness of result, but not let user to choice whice distribution mode should be use.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r715845154



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }

Review comment:
       For line 420 to 429 is trying to resolve the duplicate rows problem. If equality fields are already specified, in default, that will shuffle records by `equalityFieldIds` for unpartitioned and partitioned table. 
   
   The all cases about `Distribution Mode` discussion as follow: 
   
   Case | Stremaing Type | Distribution Mode | Primary Key | Partition Key | Validity | Shuffle by
   -- | -- | -- | -- | -- | -- | --
   1 | retract | NONE | userId | | valid | userId
   2 | retract | NONE | userId | day, hour | valid | userId
   3 | retract | HASH | userId | | valid | userId
   4 | retract | HASH | userId | day, hour | invalid | -
   5 | upsert | NONE | userId | | valid | userId
   6 | upsert | NONE | day, hour, userId | day, hour | valid | day, hour, userId
   7 | upsert | HASH | userId | | valid | userId
   8 | upsert | HASH | day, hour, userId | day, hour | valid | day, hour
   
   
   According to the solution of https://github.com/apache/iceberg/pull/2898#issuecomment-926830884, we will get the job graph as follow for all cases.
   ```
   ChangelogStream-1 ---forward---> Filter/Map-1 ---+      +---> IcebergStreamWriter-1 ---+
             ...                          ...       +-hash-+             ...              +-rebalance---> IcebergFilesCommitter
   ChangelogStream-m ---forward---> Filter/Map-m ---+      +---> IcebergStreamWriter-n ---+
   ```
   Now, we can ensure all cdc/upsert record will send to correct `IcebergStreamWriter` and no -U message will be ignored.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815542342



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer in order.
+ */
+class EqualityFieldKeySelector implements KeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final RowType flinkSchema;
+  private final Schema deleteSchema;
+
+  private transient RowDataWrapper rowDataWrapper;
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {

Review comment:
       Nit: Could we shift the `equalityFieldIds`  to be the 3rd argument ? That makes argument definition order strickly match the following assignment order.  It's a minor thing but make the code more clear.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780032820



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {

Review comment:
       nit: personally I like the if-else style. but don't need to change until someone else also thinks the same way.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1029035117


   @rdblue Any other concerns about this PR?


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812612087



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {

Review comment:
       done




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-893105327


   ping @kbendick @openinx 
   cc @rdblue @aokolnychyi @stevenzwu 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693624020



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},

Review comment:
       nit: curious why the change of 2->4?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693619492



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -140,13 +141,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case), So we need to set the map opr
+        // parallelism as it's input to keep map opr chain to input, and ensure input stream will not be rebalanced.

Review comment:
       I agree with this change. Since map operator is part of the Flink sink, it is ok to set the parallelism to whatever it is appropriate.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r779281367



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -424,6 +435,9 @@ private String operatorName(String suffix) {
         case RANGE:
           LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",

Review comment:
       this also reminds me that it might be good to add some info level logs to mention the distribution applied.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-919115588


   @Reo-LEI Thanks for the contribution, sounds like I'm late for the reviewing.  Would you like to resolve the conflicts , I think I will get this reviewing tomorrow, thanks.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719038072



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)
+            .setParallelism(input.getParallelism());

Review comment:
       I think we set different job parallelism in 'TestFlinkIcebergSinkV2' can already cover this, right?
   
   https://github.com/apache/iceberg/pull/2898/files#diff-13e2e5b52d0effe51e1b470df77cb08b5ec8cc4f3a7f0fd4e51ee212fc83f76aL130




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r809649411



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());

Review comment:
       There are quite a number of info logs that seem to repeat similar statements. Can these at least be merged somehow?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r793245548



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.util.ByteBuffers;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, ByteBuffer> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: {}", schema.findField(equalityFieldIds.get(i)));
+
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public ByteBuffer getKey(RowData row) throws Exception {

Review comment:
       this is used in the hot code path of per-record processing. small concern on the number of ByteBuffer creation/wrap, concatenation. but not sure how to make it better and maybe we don't pre-maturely optimize it until is proved to be a problem.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812751791



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+

Review comment:
       Nit: we usually left only one blank line here, right ?

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.util.ByteBuffers;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, ByteBuffer> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: {}", schema.findField(equalityFieldIds.get(i)));
+
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public ByteBuffer getKey(RowData row) throws Exception {

Review comment:
       We don't need to construct a totally new `ByteBuffer` for generating the shuffling key.  Actually we can only construct a lightweight `StructLikeWrapper` which will calculate its hashCode for shuffling purpose.   Because in the root path,  the `KeyGroupRangeAssignment` only need the key's `hashCode`. see https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java#L65 . 
   
   As this is a hot code path as @stevenzwu said, I will strongly suggest to use the shadow clone approach rather then the deep clone one. 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink 1.14: Add EqualityFieldKeySelector.

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1053853628


   Thanks @openinx  @stevenzwu @kbendick @rdblue for the reviewing! I will port this patch to flink 1.12 and 1.13 after this merged.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815542342



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer in order.
+ */
+class EqualityFieldKeySelector implements KeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final RowType flinkSchema;
+  private final Schema deleteSchema;
+
+  private transient RowDataWrapper rowDataWrapper;
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {

Review comment:
       Nit: Could we shift the `equalityFieldIds`  to be to 3rd argument ? That makes argument definition order strickly match the following assignment order.  It's a minor thing but make the code more clear.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r715795062



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       Resolve for https://github.com/apache/iceberg/issues/2918#issuecomment-926804817




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799809927



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       When users set HASH distribution, the intention is to cluster data by partition columns. if the hash distribution by partition key doesn't satisfy the required equality distribution, personally I prefer fail with an exception (instead of silently converting the behavior to equality distribution). In this case, I would say it is an incorrect config. We shouldn't set HASH distribution config at all.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718306128



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {

Review comment:
       I like this idea! I will trying to use `StructLikeWrapper` as shuffle key.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718284324



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)
+            .setParallelism(input.getParallelism());

Review comment:
       I will add am unit test to cover this.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-927258348


   Thanks @Reo-LEI for the work,  I think I will take a look today or tomarrow !   Looks like it's a PR to read & understand the whole context ! 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r724137887



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;

Review comment:
       Sounds good~

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;
+    wrapper = wrapper == null ? StructLikeWrapper.forType(deleteSchema.asStruct()) : wrapper;
+    return wrapper.set(projection.wrap(lazyRowDataWrapper().wrap(row)));

Review comment:
       LGTM.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       I got your point, but I prefer to not expose `HASH-BY-EQUALITY-COLUMNS` to user. Because user has been setting the equality columns, we should make distribution mode as `HASH-BY-EQUALITY-COLUMNS` in default to ensure the correctness of result, but not let user to choice whice distribution mode should be use.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681841099



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       The `Filter` opr is appended by flink to prevenet send null data to downstream. So the `Fliter` opr will alway exists when we use FlinkSQL to construct our job, and I think the opr name can not be change by user(so far I'm not found relevant api to rename an opr by sql).
   
   And I think we can simply set the `Filter` opr parallelism as its input and keep `Filter` chain its input opr in all case and without worrying some side-effect. Because `Filter` will not effect the input stream.
   
   Here only change the forward opr parallelism when forward opr is `Filter`, I think that is generally enough and safe condition to keep `Filter` chain its input




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697892028



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -148,10 +159,6 @@ private void testChangeLogs(List<String> equalityFieldColumns,
                               List<List<Record>> expectedRecordsPerCheckpoint) throws Exception {
     DataStream<Row> dataStream = env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO);
 
-    // Shuffle by the equality key, so that different operations of the same key could be wrote in order when

Review comment:
       Unfortunately, we could not assumes FlinkSink will got ordering input stream. Such as when using FlinkSQL, we could not easily to control network shuffle between two opr, and that will require user to have a good understanding of how FlinkSink even FlinkSQL works.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r809653028



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());

Review comment:
       > his is happening because the table is unpartitioned and there are no equality fields set. 
   
   I think what Ryan means is that if we’re logging the reason some change is occuring, we should log the full reason. An unpartitioned table with equality field ids set wouldn’t fall back to NONE but the log message tells the user only part of the reasoning so they have incomplete information on changing the behavior.
   
   And the users set writeMode, so likely some of them want to determine how to get the desired distribution mode in their job. So people will potentially think they _have_ to partition their table when they could just set equality fields and avoid having to change their data layout.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r809646241



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       Strongly agree on loudly, strongly failing (hopefully with a helpful exception) vs silently misbehaving.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815251366



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -338,6 +345,27 @@ private String operatorName(String suffix) {
       return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
     }
 
+    private List<Integer> checkAndGetEqualityFieldIds() {
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {

Review comment:
       The `identifierFieldsIds` will be a `ImmutableHashSet`, while the `equalityFieldSet` will be a `HashSet`,  the the `.equals` will always be false, right ? 
   I think it's better to change to check set size & elements for two sets.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r798729762



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());

Review comment:
       This is happening because the table is unpartitioned _and_ there are no equality fields set.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r798728391



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());

Review comment:
       Are so many info statements needed?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799494535



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());

Review comment:
       If the equality fields are not set, that is mean this table maybe don't have equality fields at all. I think we should not mention equality fields to remind user to setting the equality fields, right?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799696710



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       For HASH, we have two cases. First, the hash distribution by partition key may satisfy the required equality distribution. For example, distributing by bucket(16, id) for the equality field id satisfies the requirement because all id values are sent to the same task.
   
   If the partition distribution satisfies the equality distribution, then this should use the partition distribution. Otherwise, it should use the equality distribution to guarantee correctness. It's up to @stevenzwu and you whether you think the latter case should fail and throw an exception ("The required distribution, hash by partition key, does not correctly group the upsert key") or should warn and use the equality distribuiton.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r778573367



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StringBuilder builder = new StringBuilder("Key(");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(lazyRowDataWrapper().wrap(row)));
+    }
+    builder.append(")");
+    return builder.toString();

Review comment:
       retrun string but not `StructLikeWrapper` as key to prevent the kryo serialization issue as I mention on this [comment](https://github.com/apache/iceberg/issues/3551#issuecomment-1005360552).




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780843476



##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -84,36 +84,63 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"orc", 1, true},
-        new Object[] {"orc", 1, false},
-        new Object[] {"orc", 2, true},
-        new Object[] {"orc", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}

Review comment:
       The number of tests doesn't need to be increased by 3x. Instead, can you keep the same test cases and add distribution modes that make sense? For example, have Avro use NONE, Parquet use HASH, and ORC use RANGE.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r685772264



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       So I'm still not comfortable with relying on this `Filter` operation being appended. This assumes that Flink won't change that behavior, or that we'll catch it in between releases.
   
   Is there another way to detect the CDC case without relying on the `Filter` operation?
   
   Also, I think we'll potentially catch more cases than just CDC by relying on `Filter` (as that's a pretty generic operation name).
   
   Is there no way to detect that the source is a change data capture? Or that we more generally have a `RetractStream` (unless I'm mistaken, The +U / -U should be in play because this is a `RetractStream`).




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719110566



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;

Review comment:
       I think only `EqualityFieldKeySelector` need to project table schema to delete schema by `equalityFieldIds`, but not all `KeySelector`. Such as `PartitionKeySelector` generate key by `PartitionSpec`, but not `equalityFieldIds`.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718283099



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       Yes! `RANGE` distribution mode should use the same shuffle strategy,  I miss this case, thanks for your remind!

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)
+            .setParallelism(input.getParallelism());

Review comment:
       I will add am unit test to cover this.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;

Review comment:
       LGTM

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {

Review comment:
       I like this idea! I will trying to use `StructLikeWrapper` as shuffle key.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681115396



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}

Review comment:
       Actually, the line 94 to 101 is use  `none` as `write.distribution-mode`.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r680590851



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {
+        forwardOpr.setParallelism(forwardOpr.getInputs().get(0).getParallelism());

Review comment:
       How does setting the parallelism here interplay with the parallelism of the writer?
   
   I notice there's one other place where we set the parallelism based on some feature of the input, but we also have write parallelism that I've commented on below. Is it possible for `writeParallelism` in the below section to be different than what users have, say in the case of the CDC stream? For example, could you draw the DAG if a users input `Filter` has p=2, and then `writeParallelism` is set to `p=3`. Or is that not a possible case?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       Correct me if I'm wrong, but can't users set the name of the operators themselves?
   
   Is there a way (possibly in the Flink SDK) that we can determine if we're in CDC mode (or more generally, if we have a retract stream) that is more specific than this? I'm worried we might catch more cases here than we mean to and unnecessarily force parallelism on jobs that don't require it.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -260,18 +263,21 @@ public Builder uidPrefix(String newPrefix) {
       // Convert the requested flink table schema to flink row type.
       RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema);
 
-      // Distribute the records from input data stream based on the write.distribution-mode.
-      rowDataInput = distributeDataStream(rowDataInput, table.properties(), table.spec(), table.schema(), flinkRowType);
+      // Distribute the records from input data stream based on the write.distribution-mode and equality fields.
+      rowDataInput = distributeDataStream(rowDataInput, table.properties(), equalityFieldIds, table.spec(),
+          table.schema(), flinkRowType);
 
       // Chain the iceberg stream writer and committer operator.
       IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkRowType, equalityFieldIds);
       IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite);
 
-      this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism;
-
       SingleOutputStreamOperator<WriteResult> writerStream = rowDataInput
-          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriteResult.class), streamWriter)
-          .setParallelism(writeParallelism);
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriteResult.class), streamWriter);
+
+      if (this.writeParallelism != null) {
+        writerStream.setParallelism(writeParallelism);
+      }

Review comment:
       Is it possible for this to be `null`? I notice the null check is new.
   
   Also, in cases where you'd added `setParallelism` calls, what happens if this `writeParallelism` is also set? Is that a situation that could be encountered?

##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}

Review comment:
       Are there any tests that set any other values for `write.distribution-mode` than `hash`?
   
   Given that Flink CDC is a pretty common use case for Iceberg, It would be great to add tests here for `none` and `ordered` since this has been added to the tests as well, if possible.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -313,13 +320,21 @@ public Builder uidPrefix(String newPrefix) {
 
       switch (writeMode) {
         case NONE:
-          return input;
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, table.schema(), flinkRowType));
+          } else {
+            return input;
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
-          } else {
+          if (!partitionSpec.isUnpartitioned() && !equalityFieldIds.isEmpty()) {
+            return input.keyBy(new HybridKeySelector(partitionSpec, equalityFieldIds, iSchema, flinkRowType));
+          } else if (!partitionSpec.isUnpartitioned() && equalityFieldIds.isEmpty()) {
             return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+          } else if (partitionSpec.isUnpartitioned() && !equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, table.schema(), flinkRowType));
+          } else {
+            return input;

Review comment:
       It seems like we've gotten into a lot of `if` nesting. Would it make sense to either add a utility function, or add some method that returns more reader-friendly value from the equalityFieldIds? For me, its not as clear to see `isEmpty()` vs for example seeing `isUnpartitioned()`.
   
   Can we add a utility or a method with some name that helps developers more easily understand the different cases, especially given we have so much `if else` nesting?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/HybridKeySelector.java
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+
+public class HybridKeySelector implements KeySelector<RowData, String> {

Review comment:
       Nit but also an important question to help me understand better =)
   
   The name `HybridKeySelector` is admittedly somewhat confusing to me initially. Once I see the class definition, it's more understandable, but I do have to look through the source to see what `Hybrid` means.
   
   Which leads me to my question:
   In the case of current `HybridKeySelector`, we're using the PartitionKey and the `EqualityFieldKey`. This seems to work for `hash` distribution (at least per the updated tests), but what about `range` and `none`? Which key selectors would they use?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r798726178



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();

Review comment:
       If the equality field columns are null, then what about using the equality fields from the schema by default?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r809650596



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       It seems like a safer way to forbid using HASH distribution with equality fields. I will fix this later. 




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799520352



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();

Review comment:
       Currently, the `equalityFieldIds` value come from the `equalityFieldColumns`, and the `equalityFieldColumns` setting by user or parse from primary key of SQL. 
   
   Usually, the `equalityFieldColumns` should equal to the `identifierFieldIds` of schema, and we don't need the default value of `equalityFieldIds`. But when user remove the setting of `equalityFieldColumns` or remove parimary key from SQL, and the `identifierFieldIds` is not null. If we use `identifierFieldIds` as default value of `equalityFieldIds`, I can't imagine what's going to happen, maybe it's not what the user wants.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx merged pull request #2898: Flink 1.14: Add EqualityFieldKeySelector.

Posted by GitBox <gi...@apache.org>.
openinx merged pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898


   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815540636



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -338,6 +345,27 @@ private String operatorName(String suffix) {
       return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
     }
 
+    private List<Integer> checkAndGetEqualityFieldIds() {
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {

Review comment:
       Okay, I checked the HashSet's equal method, it does not check the concrete class type. this should be fine.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815280378



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+abstract class BaseKeySelector<I, K> implements KeySelector<I, K> {

Review comment:
       Sorry, I forgot it, I will remove this.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -338,6 +345,27 @@ private String operatorName(String suffix) {
       return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
     }
 
+    private List<Integer> checkAndGetEqualityFieldIds() {
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {

Review comment:
       That is not right, the `.equals` will compare the content of two set. I add a simple unit test for `checkAndGetEqualityFieldIds`, and the `.equals` is work.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812628062



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       Thanks @rdblue for the detailed explanation, and @kbendick and @stevenzwu for the advice!
   
   I reorganized the code of distribute. And for this case (table is partitioned and have equality fields and enabled HASH distribution) I added a check to requiring all of the partition source fields are identifier fields. Please take a look again.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815249590



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+abstract class BaseKeySelector<I, K> implements KeySelector<I, K> {

Review comment:
       So this class can be removed now ?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719901406



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",

Review comment:
       table schema can be very long and make the log difficult to read. user can always inspect table schema using tools like Spark, Trino, Flink etc. You can keep it unless more people think it is not necessary




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693335688



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       I found that we can capture the inputstream changelog mode when A is called, and then determine whether `Filter` is needed to chain its input by checking inputstream changelog mode.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697916071



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -369,6 +373,27 @@ private String operatorName(String suffix) {
         default:
           throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode);
       }
+
+      if (keySelector != null) {
+        return input.keyBy(keySelector);
+      }
+      return input;
+    }
+
+    private KeySelector<RowData, String> getKeySelector(List<Integer> equalityFieldIds, PartitionSpec partitionSpec,
+        Schema schema, RowType rowType) {
+      boolean hasPrimaryKey = equalityFieldIds != null && !equalityFieldIds.isEmpty();
+      boolean hasPartitionKey = partitionSpec != null && !partitionSpec.isUnpartitioned();
+
+      if (hasPrimaryKey && hasPartitionKey) {
+        return new CombinedKeySelector(partitionSpec, equalityFieldIds, schema, rowType);

Review comment:
       regardless the answer to the question on "partition key is a subset of equality key", we probably don't need `CombinedKeySelector `
   * yes. then equality key shuffle is equivalent to the combined key
   * no. we only need the equality key shuffle to maintain the order. combing equality key and partition key shuffle won't bring any benefit of reducing the number of data files (that hash distribution on partition key brings)




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718137998



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)
+            .setParallelism(input.getParallelism());

Review comment:
       Sounds good to me ,  do we have any unit tests to address the regression issue ? 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891584337


   > > Actually set `write.distribution-mode = hash` can not resolve this. Because CDC data shoule be distributed to primary key but not partition fields. For example, an iceberg table has equalityFields (dt, hour, id) and partition fields (dt, hour). If set `write.distribution-mode = hash`, the CDC data will be keyBy (dt, hour), and we can not ensure same id CDC data will send to same `IcebergStreamWriter`.
   > 
   > I'm a litter confused. Assume that the upstream data is orderly, if the CDC data are key by (dt, hour), the same id should be send to the same `IcebergStreamWriter`.
   > 
   > `dt=2021-08-03, hour=10, id=1` and `dt=2021-08-03, hour=10, id=2` should be the same `IcebergStreamWriter`. I think a partition is just a subset of the primary key. If anything is wrong, please let me know. Thanks!
   
   I think my example is inappropriate for CDC case which upstream of `IcebergStreamingWriter` is orderly. Because CDC data have -U record, that partition values can let record route to correct writer and delet the old record. 
   
   I think  set `write.distribution-mode = hash` will work only on iceberg table have partition fields and inpurt stream is CDC case. Onec inpurt stream is upsert stream or iceberg table is an unpartitioned table which don't have any partition fields,  set `write.distribution-mode = hash` will fail.
   
   The reason I don't agree with use `write.distribution-mode = hash` to avoid this problem is `write.distribution-mode` is use to reduce the number of small file but not to resolve row-level delete. We should not bind distribution-mode to row-level delete.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r716285220



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {
+    String[] values = new String[keySize];
+    for (int i = 0; i < keySize; i++) {
+      values[i] = accessors[i].get(lazyRowDataWrapper().wrap(row)).toString();

Review comment:
       Can `accessors[i].get(lazyRowDataWrapper().wrap(row))` return null? 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-927269706


   > Thanks @Reo-LEI for the work, I think I will take a look today or tomarrow ! Looks like it's a PR to read & understand the whole context !
   
   @openinx I think you could catch up the context through this issue https://github.com/apache/iceberg/issues/2918. :)
   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r697887709



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -83,29 +83,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},

Review comment:
       Because parallelism 2 could not cover this problem situation.  
   Suppose we have change log as follow:
   ```
   +I<1, "aaa">
   -U<1, "aaa">
   +U<1, "bbb"> 
   ```
   
   When parallelism is 2, we got DAG like that.
   ```
                                  +---> Map ---forward---> IcebergStreamWriter-1 ---+
   ChangelogStream ---rebalance---+                                                    +---rebalance---> IcebergFilesCommitter
                                  +---> Map ---forward---> IcebergStreamWriter-2 ---+
   ```
   We can know the `+I` and `+U` will be distribute to `IcebergStreamWriter-1` and `-U` will be distribute to `IcebergStreamWriter-2`. The process is wrong, but we still get the correct result, because `IcebergStreamWriter` will write pos-delete record for all has been written record in txn on `BaseEqualityDeltaWriter.write` method.
   
   If we increase the parallelism greater than 2, such as 3, we will get DAG like this.
   ```
                                  +---> Map ---forward---> IcebergStreamWriter-1 ---+
   ChangelogStream ---rebalance---+---> Map ---forward---> IcebergStreamWriter-2 ---+---rebalance---> IcebergFilesCommitter
                                  +---> Map ---forward---> IcebergStreamWriter-3 ---+
   ```
   Now `+I` will be distribute to `IcebergStreamWriter-1`, `+U` will be distribute to `IcebergStreamWriter-2` and `-U` will be distribute to `IcebergStreamWriter-3`. And will get the duplicate rows as expectations. So I increase parallelism to 4 to cover this problem situation. 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693618065



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -369,6 +373,27 @@ private String operatorName(String suffix) {
         default:
           throw new RuntimeException("Unrecognized write.distribution-mode: " + writeMode);
       }
+
+      if (keySelector != null) {
+        return input.keyBy(keySelector);
+      }
+      return input;
+    }
+
+    private KeySelector<RowData, String> getKeySelector(List<Integer> equalityFieldIds, PartitionSpec partitionSpec,
+        Schema schema, RowType rowType) {
+      boolean hasPrimaryKey = equalityFieldIds != null && !equalityFieldIds.isEmpty();
+      boolean hasPartitionKey = partitionSpec != null && !partitionSpec.isUnpartitioned();
+
+      if (hasPrimaryKey && hasPartitionKey) {
+        return new CombinedKeySelector(partitionSpec, equalityFieldIds, schema, rowType);

Review comment:
       this defeats the purpose of hash distribution, which is make sure one writer task processed data from one partition. This `CombinedKeySelector` will nullify the goal. It seems to me that hash shuffle and equality key shuffle shouldn't be enabled at the same time.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r813516316



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+

Review comment:
       Fixed.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r813517777



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.util.ByteBuffers;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, ByteBuffer> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: {}", schema.findField(equalityFieldIds.get(i)));
+
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public ByteBuffer getKey(RowData row) throws Exception {

Review comment:
       Nice catch! I have rolled back the implementation of `EqualityFieldKeySelector` to use `StructLikeWrapper` and return the `hashCode` of `Struct Like Wrapper`. Please read it again!




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r814480348



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  /**
+   * Construct the {@link StructProjection} lazily because it is not serializable.
+   */
+  protected StructProjection lazyStructProjection() {
+    if (structProjection == null) {
+      structProjection = StructProjection.create(schema, deleteSchema);
+    }
+    return structProjection;
+  }
+
+  /**
+   * Construct the {@link StructLikeWrapper} lazily because it is not serializable.
+   */
+  protected StructLikeWrapper lazyStructLikeWrapper() {
+    if (structLikeWrapper == null) {
+      structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct());
+    }
+    return structLikeWrapper;
+  }
+
+  @Override
+  public Integer getKey(RowData row) {
+    return lazyStructLikeWrapper().set(lazyStructProjection().wrap(lazyRowDataWrapper().wrap(row))).hashCode();

Review comment:
       +1. 
   
   I also have a a little concern here. For the keyBy, we are essentially doing two layers of hashCode:  equalityKeys.hashCode().hashCode(). wondering if it can cause any skewed distribution. Don't know if it is a problem or not.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815249235



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  /**
+   * Construct the {@link StructProjection} lazily because it is not serializable.
+   */
+  protected StructProjection lazyStructProjection() {
+    if (structProjection == null) {
+      structProjection = StructProjection.create(schema, deleteSchema);
+    }
+    return structProjection;
+  }
+
+  /**
+   * Construct the {@link StructLikeWrapper} lazily because it is not serializable.
+   */
+  protected StructLikeWrapper lazyStructLikeWrapper() {
+    if (structLikeWrapper == null) {
+      structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct());
+    }
+    return structLikeWrapper;
+  }
+
+  @Override
+  public Integer getKey(RowData row) {
+    return lazyStructLikeWrapper().set(lazyStructProjection().wrap(lazyRowDataWrapper().wrap(row))).hashCode();

Review comment:
       Yes, I agree with @Reo-LEI  about the `hashCode`  issues. Integer `hashCode` will always be an Integer. 
   




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780929212



##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -84,36 +84,63 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")
   public static Object[][] parameters() {
     return new Object[][] {
-        new Object[] {"avro", 1, true},
-        new Object[] {"avro", 1, false},
-        new Object[] {"avro", 2, true},
-        new Object[] {"avro", 2, false},
-        new Object[] {"orc", 1, true},
-        new Object[] {"orc", 1, false},
-        new Object[] {"orc", 2, true},
-        new Object[] {"orc", 2, false},
-        new Object[] {"parquet", 1, true},
-        new Object[] {"parquet", 1, false},
-        new Object[] {"parquet", 2, true},
-        new Object[] {"parquet", 2, false}
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE},
+
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH},
+
+        new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE},
+        new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}

Review comment:
       I think that is ok, becasue distribution modes are only related to parallelism and partition. I have remove the unnecessary test cases. 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780433476



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StructLike record = lazyRowDataWrapper().wrap(row);
+    StringBuilder builder = new StringBuilder("[");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(record));

Review comment:
       I don't think it is safe to convert to String implicitly because some values don't implement toString for a string representation. I think that you'll get byte[] or ByteBuffer and that would cause you to get different keys for equal fixed or binary values.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI edited a comment on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI edited a comment on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1043819927


   > @Reo-LEI @rdblue hi,boys,We are currently using the flink upsert function. In which version is this PR planned to be released?
   
   I will adressing the comments in this weeken. Thank you for your attention! @liubo1022126


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681121716



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -260,18 +263,21 @@ public Builder uidPrefix(String newPrefix) {
       // Convert the requested flink table schema to flink row type.
       RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema);
 
-      // Distribute the records from input data stream based on the write.distribution-mode.
-      rowDataInput = distributeDataStream(rowDataInput, table.properties(), table.spec(), table.schema(), flinkRowType);
+      // Distribute the records from input data stream based on the write.distribution-mode and equality fields.
+      rowDataInput = distributeDataStream(rowDataInput, table.properties(), equalityFieldIds, table.spec(),
+          table.schema(), flinkRowType);
 
       // Chain the iceberg stream writer and committer operator.
       IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkRowType, equalityFieldIds);
       IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite);
 
-      this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism;
-
       SingleOutputStreamOperator<WriteResult> writerStream = rowDataInput
-          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriteResult.class), streamWriter)
-          .setParallelism(writeParallelism);
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriteResult.class), streamWriter);
+
+      if (this.writeParallelism != null) {
+        writerStream.setParallelism(writeParallelism);
+      }

Review comment:
       That would be null if use not set, if `writeParallelism` is null, the `writerStream` would use flink job default parallelism, or else use `writeParallelism` specific value. This `writeParallelism` setting only effect to `IcebergStreamingWriter` parallelism, I think that would not bring other side effect. 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681846886



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/HybridKeySelector.java
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+
+public class HybridKeySelector implements KeySelector<RowData, String> {

Review comment:
       I agree `HybridKeySelector` is not easy to understand. So I will rename it to `CombinedKeySelector`.
   
   I think distribution mode and key selector is not one-to-one correspondence, different distribution mode will use 
   `EqualityFieldKeySelector` or `PartitionKeySelector` or `CombinedKeySelector`. And `range` mode will not be consider here, cuz flink not support `range` mode yet.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI edited a comment on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI edited a comment on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-919277576


   > @Reo-LEI Thanks for the contribution, sounds like I'm late for the reviewing. Would you like to resolve the conflicts , I think I will get this reviewing tomorrow, thanks.
   
   @openinx Thanks for your reply. Since I discuss this PR with @kbendick  @stevenzwu, I think the implementation of setting `Filter` opr parallelism and use `CombinedKeySelector` to key by equality key and partition key maybe not a good way to reslove this problem. I'm plan to refactor this on this weeken, and maybe you could review this PR after that. :)


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-890624681


   Also, as my Flink is a little rusty compared to my Spark knowledge, can you please open an issue (or link it to your commit)? I can help tag some relevant people there if we don't hear from openinx after a bit here =)


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick edited a comment on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick edited a comment on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-890624681


   Also, as my Flink is a little rusty compared to my Spark knowledge, can you please open an issue (or link it to your commit)? I can help tag some relevant people there if we don't hear from openinx after a bit here as I'd love to have other more knowledgable folks than myself contribute to the discussion. =)
   
   EDIT - Finally, we've talked a lot about the CDC case that it map side only (though with a potential rebalance or hash inside of it), but can users still get themselves into an unhappy case if they have a retract stream that they re-key or something themselves (say they do some ETL on a CDC stream)?


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r779281098



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -424,6 +435,9 @@ private String operatorName(String suffix) {
         case RANGE:
           LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",

Review comment:
       we need to move this log line into an else block




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799482202



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       I discuss the [approach ](https://github.com/apache/iceberg/pull/2898#discussion_r693618065) of combined key with steven before. And I list all cases of distribution in [here](https://github.com/apache/iceberg/pull/2898#discussion_r715845154). 
   
   For this case(table is partitioned and have equality fields and enabled `HASH` distribution), I think we should demand the partition key must be a part of equality key, and we could distribute rows by equality key. Otherwise, we should raise an error and not allow use `HASH` distribution with equality fields.
   




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812616420



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());

Review comment:
       I rephrased the log and added the full reason. Please take a look again.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r813516201



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+

Review comment:
       Fixed.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r815251386



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -338,6 +345,27 @@ private String operatorName(String suffix) {
       return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
     }
 
+    private List<Integer> checkAndGetEqualityFieldIds() {
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {

Review comment:
       The `identifierFieldsIds` will be a `ImmutableHashSet`, while the `equalityFieldSet` will be a `HashSet`,  the the `.equals` will always be false, right ? 
   I think it's better to change to check set size & elements for two sets.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-948238409


   @openinx @stevenzwu @kbendick Do you have another other concerns for this PR? Maybe we could merge this PR to fix this problem. 😄 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780843101



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StructLike record = lazyRowDataWrapper().wrap(row);
+    StringBuilder builder = new StringBuilder("[");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(record));

Review comment:
       No, we don't have nested types in equality fields.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719900158



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;

Review comment:
       sorry, I meant `lazyStructProjection()` in this `EqualityFieldKeySelector ` class, not the base class




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718283099



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       Yes! `RANGE` distribution mode should use the same shuffle strategy,  I miss this case, thanks for your remind!




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719050707



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;
+    wrapper = wrapper == null ? StructLikeWrapper.forType(deleteSchema.asStruct()) : wrapper;
+    return wrapper.set(projection.wrap(lazyRowDataWrapper().wrap(row)));

Review comment:
       why does the return type need to be `StructLikeWrapper`? Can we directly return `projection.wrap(lazyRowDataWrapper().wrap(row))`?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r798731781



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       Why not use both partition and equality field? Why is it okay to ignore equality fields for this case?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-926858684


   I have completed the refactoring of this PR, Could you help review this PR again? @kbendick @stevenzwu @openinx 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-891165008


   > EDIT - Finally, we've talked a lot about the CDC case that it map side only (though with a potential rebalance or hash inside of it), but can users still get themselves into an unhappy case if they have a retract stream that they re-key or something themselves (say they do some ETL on a CDC stream)?
   
   Could you give me some examples for this case? For upsert and retract stream, I think these streams should keyBy `equalityFields`(primary key) to keep update the same row with same pk. 
   


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-890517576


   Could you take look of this? @openinx 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719108932



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;
+    wrapper = wrapper == null ? StructLikeWrapper.forType(deleteSchema.asStruct()) : wrapper;
+    return wrapper.set(projection.wrap(lazyRowDataWrapper().wrap(row)));

Review comment:
       Because `StructProjection` don't have `hashCode` method, we need to depend on `StructLikeWrapper` to offer `hashCode` method used by `KeySelector`.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718132230



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;

Review comment:
       I don't think we need to maintain this `keySize` to be a local variable.  The `accessors.length` is good enough to get the length in the following `getKey` method.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r724139446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;
+    wrapper = wrapper == null ? StructLikeWrapper.forType(deleteSchema.asStruct()) : wrapper;
+    return wrapper.set(projection.wrap(lazyRowDataWrapper().wrap(row)));

Review comment:
       LGTM.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r813516316



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/BaseKeySelector.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+

Review comment:
       Fixed.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r779278816



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StringBuilder builder = new StringBuilder("Key(");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(lazyRowDataWrapper().wrap(row)));

Review comment:
       nit: we can `wrap(row)` once outside the for loop




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799192961



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       hash distribution was trying to bring all rows for the same partition value to a single writer. if we include equality field, that would defeat the purpose. On the other hand, if we don't include equality field, we would still suffer the issue that this PR is trying to address. Maybe with equality field, we shouldn't use `HASH` distribution?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780792615



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StructLike record = lazyRowDataWrapper().wrap(row);
+    StringBuilder builder = new StringBuilder("[");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(record));

Review comment:
       That is make sense, I will serialize the key as ByteBuffer. And I want to know are we allow use NestedType(List/Map/Struct) as equality field? If we allow this, I need a little more effort to support this.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-893772869


   This PR should link to this issue: https://github.com/apache/iceberg/issues/2918


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r716284472



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;

Review comment:
       nit: use primitive type?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r685777739



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {

Review comment:
       I guess this could also be an `UpsertStream`.
   
   Would it make more sense to check if the stream is either an `UpsertStream` or a `RetractStream`? Checking for `Filter` seems brittle to me and as though it likely casts a very wide net.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r681128531



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {
+        forwardOpr.setParallelism(forwardOpr.getInputs().get(0).getParallelism());

Review comment:
       `Filter` parallelism different than `IcebergStreamingWriter` is possible and common. Here setting `Filter` parallelism as it's input is want to let `Filter` chain to it's Input, and keep input stream data original order. `writeParallelism` will setting the `IcebergStreaming` parallelism, that will only effect on write performance. 
   
   If `equalityFieldColumns` has been setting, that is mean data stream will be `keyBy` by `equalityFieldColumns` between `Filter` and `IcebergStreamingWriter`. If `equalityFieldColumns` is null, that will be `rebalance` between `Filter` and `IcebergStreamingWriter`.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-893105327


   ping @kbendick @openinx 
   cc @rdblue @aokolnychyi @stevenzwu 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r814680116



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;

Review comment:
       OK,I will remove the `BaseKeySelector` until we got more other key selector and we need this.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field

Review comment:
       done

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  /**
+   * Construct the {@link StructProjection} lazily because it is not serializable.
+   */
+  protected StructProjection lazyStructProjection() {
+    if (structProjection == null) {
+      structProjection = StructProjection.create(schema, deleteSchema);
+    }
+    return structProjection;
+  }
+
+  /**
+   * Construct the {@link StructLikeWrapper} lazily because it is not serializable.
+   */
+  protected StructLikeWrapper lazyStructLikeWrapper() {
+    if (structLikeWrapper == null) {
+      structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct());
+    }
+    return structLikeWrapper;
+  }
+
+  @Override
+  public Integer getKey(RowData row) {
+    return lazyStructLikeWrapper().set(lazyStructProjection().wrap(lazyRowDataWrapper().wrap(row))).hashCode();

Review comment:
       > Nit: It more clear for me to understand the unfolded code like the following
   
   Done. 
   
   > For the keyBy, we are essentially doing two layers of hashCode: equalityKeys.hashCode().hashCode(). wondering if it can cause any skewed distribution.
   
   I think we shoult not worry about the second `hashCode()`, because Interger's `hashCode` is the value itself.
   
   However, `keyBy` with equailty fields does have skewed data in some cases, but I think we can avoid the skewed data by setting the equailty fields, using the equailty fields as a partition, and then turning on the `HASH` distribution mode (as Ryan said, make sure that all of the partition source fields are identifier fields, we can distribute data by partition key).
   
   For example, if the equailty fields of a table is `id` and `type`, we can partition the `id`, such as `bucket(id, 16)`, and then turn on the `HASH distribution mode`, and the data will be distributed by the partition key  `bucket(id, 16)`. And we avoid the skewed data.
   

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -146,13 +148,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as its input to keep map operator chaining its input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)

Review comment:
       Currently, many test cases of `TestFlinkIcebergSinkV2` are depend on this config. If we remove this, these test case would be faild. I think these test cases has cover this change.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -295,15 +298,34 @@ public Builder setSnapshotProperty(String property, String value) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {
+          LOG.warn("The configured equality field columns are not match with the identifier fields of schema, " +

Review comment:
       done

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -295,15 +298,34 @@ public Builder setSnapshotProperty(String property, String value) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {
+          LOG.warn("The configured equality field columns are not match with the identifier fields of schema, " +
+              "use job specified equality field columns as the equality fields by default.");
+        }
+        equalityFieldIds = Lists.newArrayList(equalityFieldSet);
+      }

Review comment:
       done

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -422,21 +436,50 @@ private String operatorName(String suffix) {
         writeMode = distributionMode;
       }
 
+      LOG.info("Write distribution mode is '{}'", writeMode.modeName());
       switch (writeMode) {
         case NONE:
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set");
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
+          if (equalityFieldIds.isEmpty()) {
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                  "and table is unpartitioned");
+              return input;
+            } else {
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           } else {
-            return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                  "and table is unpartitioned");
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            } else {
+              for (PartitionField partitionField : partitionSpec.fields()) {
+                Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()),
+                    "In 'hash' distribution mode with equality fields set, partition field '%s' " +
+                        "should be included in equality fields: '%s'", partitionField, equalityFieldColumns);
+              }
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           }
 
         case RANGE:
-          LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",
-              WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");

Review comment:
       done

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -422,21 +436,50 @@ private String operatorName(String suffix) {
         writeMode = distributionMode;
       }
 
+      LOG.info("Write distribution mode is '{}'", writeMode.modeName());
       switch (writeMode) {
         case NONE:
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set");
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
+          if (equalityFieldIds.isEmpty()) {
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                  "and table is unpartitioned");
+              return input;
+            } else {
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           } else {
-            return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                  "and table is unpartitioned");
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            } else {
+              for (PartitionField partitionField : partitionSpec.fields()) {
+                Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()),
+                    "In 'hash' distribution mode with equality fields set, partition field '%s' " +
+                        "should be included in equality fields: '%s'", partitionField, equalityFieldColumns);
+              }
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           }
 
         case RANGE:
-          LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",
-              WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");

Review comment:
       done

##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -84,36 +84,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")

Review comment:
       done

##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -155,10 +159,6 @@ private void testChangeLogs(List<String> equalityFieldColumns,
                               List<List<Record>> expectedRecordsPerCheckpoint) throws Exception {
     DataStream<Row> dataStream = env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO);
 
-    // Shuffle by the equality key, so that different operations of the same key could be wrote in order when
-    // executing tasks in parallel.
-    dataStream = dataStream.keyBy(keySelector);

Review comment:
       Yes, because we ensure the input will not be disorder in `FlinkSink.forMapperOutputType` and data will be correctly  distributed by equaily fields or partition fields in different distribution mode. So we can remove this in unittest




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r814414515



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;

Review comment:
       As this `schema` has been defined in its parent class, right ?  Then here we don't need to add an extra `schema` field in this children class. Personally,  I think the can just make all the logics from `BaseKeySelector` into the `EqualityFieldKeySelector`, because the current abstraction of `BaseKeySelector` seems only provide a `lazyRowDataWrapper` method.   Just removing the `BaseKeySelector` looks more clear for me.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection structProjection;
+  private transient StructLikeWrapper structLikeWrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  /**
+   * Construct the {@link StructProjection} lazily because it is not serializable.
+   */
+  protected StructProjection lazyStructProjection() {
+    if (structProjection == null) {
+      structProjection = StructProjection.create(schema, deleteSchema);
+    }
+    return structProjection;
+  }
+
+  /**
+   * Construct the {@link StructLikeWrapper} lazily because it is not serializable.
+   */
+  protected StructLikeWrapper lazyStructLikeWrapper() {
+    if (structLikeWrapper == null) {
+      structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct());
+    }
+    return structLikeWrapper;
+  }
+
+  @Override
+  public Integer getKey(RowData row) {
+    return lazyStructLikeWrapper().set(lazyStructProjection().wrap(lazyRowDataWrapper().wrap(row))).hashCode();

Review comment:
       Nit:  It more clear for me to understand the unfolded code like the following: 
   
   ```java
     public Integer getKey(RowData row) {
       RowDataWrapper rowDataWrapper = lazyRowDataWrapper().wrap(row);
       StructProjection structProjection = lazyStructProjection().wrap(rowDataWrapper);
       StructLikeWrapper structLikeWrapper = lazyStructLikeWrapper().set(structProjection);
   
       return structLikeWrapper.hashCode();
     }
   ```

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -422,21 +436,50 @@ private String operatorName(String suffix) {
         writeMode = distributionMode;
       }
 
+      LOG.info("Write distribution mode is '{}'", writeMode.modeName());
       switch (writeMode) {
         case NONE:
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set");
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
+          if (equalityFieldIds.isEmpty()) {
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                  "and table is unpartitioned");
+              return input;
+            } else {
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           } else {
-            return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                  "and table is unpartitioned");
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            } else {
+              for (PartitionField partitionField : partitionSpec.fields()) {
+                Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()),
+                    "In 'hash' distribution mode with equality fields set, partition field '%s' " +
+                        "should be included in equality fields: '%s'", partitionField, equalityFieldColumns);
+              }
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           }
 
         case RANGE:
-          LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",
-              WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");

Review comment:
       Nit: Keep to  use the `WRITE_DISTRIBUTION_MODE` and `DistributionMode.Range `for refactoring purpose.

##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -84,36 +84,39 @@
   private final FileFormat format;
   private final int parallelism;
   private final boolean partitioned;
+  private final String distributionMode;
 
   private StreamExecutionEnvironment env;
   private TestTableLoader tableLoader;
 
-  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}")
+  @Parameterized.Parameters(name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, Distribution={3}")

Review comment:
       Nit: Use the full name `WriteDistributionMode` pls.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field

Review comment:
       Nit:  I think add the `will be emitted to same writer in order` will make more clear.  The next sentence about `That can prevent ... will take effect` can be removed.  

##########
File path: flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -155,10 +159,6 @@ private void testChangeLogs(List<String> equalityFieldColumns,
                               List<List<Record>> expectedRecordsPerCheckpoint) throws Exception {
     DataStream<Row> dataStream = env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO);
 
-    // Shuffle by the equality key, so that different operations of the same key could be wrote in order when
-    // executing tasks in parallel.
-    dataStream = dataStream.keyBy(keySelector);

Review comment:
       Then the argument `keySelector` in testChangeLogs method can be removed now ? 

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -422,21 +436,50 @@ private String operatorName(String suffix) {
         writeMode = distributionMode;
       }
 
+      LOG.info("Write distribution mode is '{}'", writeMode.modeName());
       switch (writeMode) {
         case NONE:
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set");
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
 
         case HASH:
-          if (partitionSpec.isUnpartitioned()) {
-            return input;
+          if (equalityFieldIds.isEmpty()) {
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                  "and table is unpartitioned");
+              return input;
+            } else {
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           } else {
-            return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            if (partitionSpec.isUnpartitioned()) {
+              LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                  "and table is unpartitioned");
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            } else {
+              for (PartitionField partitionField : partitionSpec.fields()) {
+                Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()),
+                    "In 'hash' distribution mode with equality fields set, partition field '%s' " +
+                        "should be included in equality fields: '%s'", partitionField, equalityFieldColumns);
+              }
+              return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType));
+            }
           }
 
         case RANGE:
-          LOG.warn("Fallback to use 'none' distribution mode, because {}={} is not supported in flink now",
-              WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
-          return input;
+          if (equalityFieldIds.isEmpty()) {
+            LOG.warn("Fallback to use 'none' distribution mode, because there are no equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");
+            return input;
+          } else {
+            LOG.info("Distribute rows by equality fields, because there are equality fields set " +
+                "and write.distribution-mode=range is not supported yet in flink");

Review comment:
       Ditto.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -295,15 +298,34 @@ public Builder setSnapshotProperty(String property, String value) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {
+          LOG.warn("The configured equality field columns are not match with the identifier fields of schema, " +

Review comment:
       Nit: `match` -> `matched`.   It's better to log the two different field id set in this message.

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -146,13 +148,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as its input to keep map operator chaining its input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)

Review comment:
       Nice catch.   Is possible for me add an unit test to address this catching ? 

##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -295,15 +298,34 @@ public Builder setSnapshotProperty(String property, String value) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds());
+      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
+        Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size());
+        for (String column : equalityFieldColumns) {
+          org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column);
+          Preconditions.checkNotNull(field, "Missing required equality field column '%s' in table schema %s",
+              column, table.schema());
+          equalityFieldSet.add(field.fieldId());
+        }
+
+        if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) {
+          LOG.warn("The configured equality field columns are not match with the identifier fields of schema, " +
+              "use job specified equality field columns as the equality fields by default.");
+        }
+        equalityFieldIds = Lists.newArrayList(equalityFieldSet);
+      }

Review comment:
       Let's make this into a separate small method named `checkAndGetEqualityFieldIds`  to make this `build` core logic more clear.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693623631



##########
File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java
##########
@@ -148,10 +159,6 @@ private void testChangeLogs(List<String> equalityFieldColumns,
                               List<List<Record>> expectedRecordsPerCheckpoint) throws Exception {
     DataStream<Row> dataStream = env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO);
 
-    // Shuffle by the equality key, so that different operations of the same key could be wrote in order when

Review comment:
       maybe this unit test demonstrates the original intention that FlinkSink assumes certainly ordering of input stream and it is not the responsibility of the FlinkSink to correct any ordering problem.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] coolderli commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
coolderli commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-890282150


   I think this may be avoided by setting `write.distribution-mode` to `hash`.
   https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java#L322


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r693619096



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {
+        forwardOpr.setParallelism(forwardOpr.getInputs().get(0).getParallelism());

Review comment:
       This is what worries me a little. this might make sense for the filter operator for the CDC stream. but is it always a good thing for other scenarios for the Flink Iceberg sink to change the parallelism of an upstream operator (which doesn't below to the Iceberg sink)?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r724137887



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.StructProjection;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, StructLikeWrapper> {
+
+  private final Schema schema;
+  private final Schema deleteSchema;
+
+  private transient StructProjection projection;
+  private transient StructLikeWrapper wrapper;
+
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.schema = schema;
+    this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
+  }
+
+  @Override
+  public StructLikeWrapper getKey(RowData row) {
+    // lazily construct because projection and wrapper are not serializable.
+    projection = projection == null ? StructProjection.create(schema, deleteSchema) : projection;

Review comment:
       Sounds good~




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r719902732



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       I am wondering if we can have explicit `DistributionMode` to shuffle by equality columns. Current `HASH` mode is more like `HASH-BY-PARTITION-COLUMNS`. We can have a `HASH-BY-EQUALITY-COLUMNS`. We can discuss the names, but you should get my idea. 




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718139919



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       As the `RANGE` distribution mode will also fallback to use `NONE` mode,  shouldn't we also use the same shuffle strategy for it ?




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718135802



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {

Review comment:
       Should we convert all those rows into a `String` ?  I think that will introduce much serialize cost, right ? How about using the `StructLikeWrapper` if the flink backend won't reference the generated 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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-902190691


   @Reo-LEI I left a comment in the issue #2918. Let's continue the problem discussion there. 
   
   You probably want to rebase the PR properly. right now, it shows 1,000+ files modified.


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r718132230



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;

Review comment:
       I don't think we need to maintain this `keySize` to be a local variable.  The `accessors.length` is good enough to get the length in the following `getKey` method.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private static final String SEPARATOR = "-";
+  private final Integer keySize;
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+    this.keySize = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, keySize);
+
+    for (int i = 0; i < keySize; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) {

Review comment:
       Should we convert all those rows into a `String` ?  I think that will introduce much serialize cost, right ? How about using the `StructLikeWrapper` if the flink backend won't reference the generated keys.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.

Review comment:
       NIt: `it's` -> `its` ?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -144,13 +144,14 @@ private Builder forRowData(DataStream<RowData> newRowDataInput) {
                                             MapFunction<T, RowData> mapper,
                                             TypeInformation<RowData> outputType) {
       this.inputCreator = newUidPrefix -> {
+        // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we need to set the parallelism
+        // of map operator same as it's input to keep map operator chaining it's input, and avoid rebalanced by default.
+        SingleOutputStreamOperator<RowData> inputStream = input.map(mapper, outputType)
+            .setParallelism(input.getParallelism());

Review comment:
       Sounds good to me ,  do we have any unit tests to address the regression issue ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       As the `RANGE` distribution mode will also fallback to use `NONE` mode,  shouldn't we also use the same shuffle strategy for it ?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,10 +417,16 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));

Review comment:
       See: https://github.com/apache/iceberg/pull/2898/files#diff-a26cb8b3f0f5b0e6a9f110bb0b3fdd0ed625bfc9faf27e248f6446818cad4d69R436




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r715795199



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
##########
@@ -59,12 +60,22 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
         .map(UniqueConstraint::getColumns)
         .orElseGet(ImmutableList::of);
 
-    return (DataStreamSinkProvider) dataStream -> FlinkSink.forRowData(dataStream)
-        .tableLoader(tableLoader)
-        .tableSchema(tableSchema)
-        .equalityFieldColumns(equalityColumns)
-        .overwrite(overwrite)
-        .build();
+    return (DataStreamSinkProvider) dataStream ->  {
+      // For CDC case in FlinkSQL, change log will be rebalanced(default partition strategy) distributed to Filter opr
+      // when set job default parallelism greater than 1. That will make change log data disorder and produce a wrong
+      // result for iceberg(e.g. +U comes before -U). Here try to specific the Filter opr parallelism same as it's
+      // input to keep Filter chaining it's input and avoid rebalance.
+      Transformation<?> forwardOpr = dataStream.getTransformation();
+      if (forwardOpr.getName().equals("Filter") && forwardOpr.getInputs().size() == 1) {
+        forwardOpr.setParallelism(forwardOpr.getInputs().get(0).getParallelism());

Review comment:
       Resolve for https://github.com/apache/iceberg/issues/2918#issuecomment-926804817




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r799487432



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());

Review comment:
       Yes, I agree with steven on this [discussion ](https://github.com/apache/iceberg/pull/2898#discussion_r779281367) that we should indicate we distribute rows by what key in what distribution mode.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r809646241



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       Strongly agree on loudly, strongly failing (hopefully with a helpful exception) vs silently misbehaving by going against the user’s configuration in a pretty heavy way.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r810411948



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -412,18 +417,38 @@ private String operatorName(String suffix) {
 
       switch (writeMode) {
         case NONE:
+          if (!equalityFieldIds.isEmpty()) {
+            LOG.info("Distribute rows by equality fields in '{}' distribution mode", DistributionMode.NONE.modeName());
+            return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+          }
+
           return input;
 
         case HASH:
           if (partitionSpec.isUnpartitioned()) {
+            if (!equalityFieldIds.isEmpty()) {
+              LOG.info("Distribute rows by equality fields in '{}' distribution mode, because table is unpartitioned",
+                  DistributionMode.HASH.modeName());
+              return input.keyBy(new EqualityFieldKeySelector(equalityFieldIds, iSchema, flinkRowType));
+            }
+
+            LOG.warn("Fallback to use '{}' distribution mode, because table is unpartitioned",
+                DistributionMode.NONE.modeName());
             return input;
           } else {
+            LOG.info("Distribute rows by partition fields in '{}' distribution mode", DistributionMode.HASH.modeName());

Review comment:
       @Reo-LEI, I don't think that's accurate. The requirement is to detect whether the equality distribution is satisfied by the hash distribution. If it is, then you can use the hash distribution. If not, then fail.
   
   I think that check should be whether there are any partition fields that use a source column that is not part of the identifier fields. For example, a table with `type`, `id`, and `data` and a simple key `id` requires any partition transform that uses `id`, like `bucket(256, id)` or `truncate(id, 10)`. But if there is an additional field like `data` then we can't guarantee that all `id` values go to the same data partition.
   
   Another case that does work is when the table identifier is made from both `type` and `id`. In that case, partitioning by either `type` or `id` individually works. Same with partitioning by `type` and `id`. But again, partitioning by data ruins the distribution.
   
   So the logic should be: make sure that the partition spec includes at least one identifier field column and that all of the partition source fields are identifier fields.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1043819927


   > @Reo-LEI @rdblue hi,boys,We are currently using the flink upsert function. In which version is this PR planned to be released?
   
   I will adressing comments in this weeken. Thank you for your attention! @liubo1022126


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r812612384



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -283,15 +284,27 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
+      // Find out the equality field id list based on the user-provided equality field column names.
+      List<Integer> equalityFieldIds = Lists.newArrayList();

Review comment:
       done




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780433476



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StructLike record = lazyRowDataWrapper().wrap(row);
+    StringBuilder builder = new StringBuilder("[");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(record));

Review comment:
       I don't think it is safe to convert to String implicitly because some values don't implement toString for a string representation. I think that you'll get byte[] or ByteBuffer and that would cause you to get different keys for equal fixed or binary values.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#issuecomment-1005419591


   I fix the kryo serialization problem of #3551, dose anybody can take an other look of this PR? If there are no other concerns about this PR, I think we can merge this. @rdblue @openinx @jackye1995 @stevenzwu @kbendick 


-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Reo-LEI commented on a change in pull request #2898: Flink: Fix duplicate rows when sync CDC data by FlinkSQL

Posted by GitBox <gi...@apache.org>.
Reo-LEI commented on a change in pull request #2898:
URL: https://github.com/apache/iceberg/pull/2898#discussion_r780931114



##########
File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.iceberg.flink.sink;
+
+import java.lang.reflect.Array;
+import java.util.List;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.Accessor;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record will be emitted to
+ * same writer. That can prevent create duplicate record when insert and delete one row which have same equality field
+ * values on different writer in one transaction, and guarantee pos-delete will take effect.
+ */
+class EqualityFieldKeySelector extends BaseKeySelector<RowData, String> {
+
+  private final Accessor<StructLike>[] accessors;
+
+  @SuppressWarnings("unchecked")
+  EqualityFieldKeySelector(List<Integer> equalityFieldIds, Schema schema, RowType flinkSchema) {
+    super(schema, flinkSchema);
+
+    int size = equalityFieldIds.size();
+    this.accessors = (Accessor<StructLike>[]) Array.newInstance(Accessor.class, size);
+    for (int i = 0; i < size; i++) {
+      Accessor<StructLike> accessor = schema.accessorForField(equalityFieldIds.get(i));
+      Preconditions.checkArgument(accessor != null,
+          "Cannot build accessor for field: " + schema.findField(equalityFieldIds.get(i)));
+      accessors[i] = accessor;
+    }
+  }
+
+  @Override
+  public String getKey(RowData row) throws Exception {
+    StructLike record = lazyRowDataWrapper().wrap(row);
+    StringBuilder builder = new StringBuilder("[");
+    for (int i = 0; i < accessors.length; i++) {
+      if (i != 0) {
+        builder.append(",");
+      }
+      builder.append(accessors[i].get(record));

Review comment:
       Great! So I can keep to use `Conversions` to serialize the key as ByteBuffer.




-- 
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@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org