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: [email protected]

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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to