You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/10/25 17:09:47 UTC

[GitHub] [hudi] nsivabalan opened a new pull request #2206: Adding dedup support for Bulk Insert w/ Rows

nsivabalan opened a new pull request #2206:
URL: https://github.com/apache/hudi/pull/2206


   ## What is the purpose of the pull request
   
   Adding dedup support for Bulk Insert w/ Rows
   
   ## Brief change log
   
     - Adding dedup support for Bulk Insert w/ Rows
     - Introduced an interface called PreCombineRow to reduce two Rows to one for deduping purposes. 
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
   
     - *Added TestHoodieDatasetBulkInsertHelper to verify the change.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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

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



[GitHub] [hudi] hudi-bot commented on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


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


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

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



[GitHub] [hudi] vinothchandar commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664284205



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -98,6 +98,8 @@
   public static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
   public static final String COMBINE_BEFORE_DELETE_PROP = "hoodie.combine.before.delete";
   public static final String DEFAULT_COMBINE_BEFORE_DELETE = "true";
+  public static final String COMBINE_BEFORE_BULK_INSERT_PROP = "hoodie.combine.before.bulk.insert";

Review comment:
       all these configs need to be redone based ConfigProperty/HoodieConfig

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();

Review comment:
       why the singleton etc? Can't we just use a static method?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+

Review comment:
       nit: extra line?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
##########
@@ -96,9 +97,15 @@
                 functions.lit("").cast(DataTypes.StringType))
             .withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
                 functions.lit("").cast(DataTypes.StringType));
+
+    Dataset<Row> dedupedDf = rowDatasetWithHoodieColumns;
+    if (config.shouldCombineBeforeBulkInsert()) {

Review comment:
       I understand that the new config is just used here as of this PR. but from an user standpoint, on the non-row writer path, combine.before.insert was controlling this already. We should just make it consistent. 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -306,6 +308,10 @@ public boolean shouldCombineBeforeInsert() {
     return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
   }
 
+  public boolean shouldCombineBeforeBulkInsert() {

Review comment:
       So far, we have used one config `combine.before.insert` to control it for both insert and bulk_insert. Can we keep it the same way? Otherwise, wont it be backwards incompatible, ie a user can be expecting the combine.before.insert continue to take effect for bulk_insert as well and it won't be the case?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(
+        (MapFunction<Row, String>) value ->
+            isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
+                (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
+        .reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
+          if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
+            return v1;
+          } else {
+            return v2;
+          }
+            }
+        ).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
+  }
+
+  private ExpressionEncoder getEncoder(StructType schema) {
+    List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
+        .map(Attribute::toAttribute).collect(Collectors.toList());
+    return RowEncoder.apply(schema)

Review comment:
       have you tested with both Spark 2 and 3?  Some of these classes can be different and actually fail?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(

Review comment:
       lets use reduceByKey(), which we use for RDD path? groupByKey() can hog memory. 

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -333,7 +334,9 @@ private[hudi] object HoodieSparkSqlWriter {
     log.info(s"Registered avro schema : ${schema.toString(true)}")
     val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
     val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
+    val isGlobalIndex = SparkHoodieIndex.createIndex(writeConfig).isGlobal

Review comment:
       do we need to create the index to really check if its Global? Wondering if there are simpler means. (may be not, given we support even user defined indexes) 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (51ccc2d) into [master](https://codecov.io/gh/apache/hudi/commit/ea9e5d0e8b7557ef82631ac173d67f15bad13690?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ea9e5d0) will **decrease** coverage by `24.55%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     27.42%   2.87%   -24.56%     
   + Complexity     1285      82     -1203     
   ============================================
     Files           381     281      -100     
     Lines         15098   11618     -3480     
     Branches       1303     950      -353     
   ============================================
   - Hits           4141     334     -3807     
   - Misses        10659   11258      +599     
   + Partials        298      26      -272     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudiclient | `0.00% <ø> (-21.07%)` | :arrow_down: |
   | hudisync | `5.28% <ø> (ø)` | |
   | hudiutilities | `9.26% <ø> (-49.37%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../hudi/utilities/callback/SourceCommitCallback.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2NhbGxiYWNrL1NvdXJjZUNvbW1pdENhbGxiYWNrLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [137 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [ea9e5d0...51ccc2d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (51ccc2d) into [master](https://codecov.io/gh/apache/hudi/commit/ea9e5d0e8b7557ef82631ac173d67f15bad13690?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ea9e5d0) will **decrease** coverage by `20.20%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2206       +/-   ##
   =============================================
   - Coverage     47.61%   27.41%   -20.21%     
   + Complexity     5495     1285     -4210     
   =============================================
     Files           929      381      -548     
     Lines         41240    15106    -26134     
     Branches       4135     1303     -2832     
   =============================================
   - Hits          19637     4141    -15496     
   + Misses        19859    10667     -9192     
   + Partials       1744      298     -1446     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `21.05% <0.00%> (-13.56%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.62% <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/java/org/apache/hudi/index/SparkHoodieIndex.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW5kZXgvU3BhcmtIb29kaWVJbmRleC5qYXZh) | `56.52% <0.00%> (-30.15%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/hudi/metrics/MetricsReporter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/metrics/MetricsReporterType.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/client/bootstrap/BootstrapMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC9ib290c3RyYXAvQm9vdHN0cmFwTW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC91dGlscy9Db25jYXRlbmF0aW5nSXRlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/hudi/config/HoodieMetricsPrometheusConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NvbmZpZy9Ib29kaWVNZXRyaWNzUHJvbWV0aGV1c0NvbmZpZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2V4ZWN1dGlvbi9idWxraW5zZXJ0L0J1bGtJbnNlcnRTb3J0TW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [615 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [ea9e5d0...51ccc2d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 031b8fa2a947f69815bce9fa181dc98dd972d07e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202) 
   * 80ae3446670e4012ef3896477964bb916b71a864 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0b5cdce9cea59681d7e604c6f6706a4e96a19861 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749) 
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80ae3446670e4012ef3896477964bb916b71a864 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748) 
   * 0b5cdce9cea59681d7e604c6f6706a4e96a19861 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749) 
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ad1d2d1) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **decrease** coverage by `42.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     45.85%   2.87%   -42.98%     
   + Complexity     4699      82     -4617     
   ============================================
     Files           828     280      -548     
     Lines         37742   11615    -26127     
     Branches       3780     950     -2830     
   ============================================
   - Hits          17305     334    -16971     
   + Misses        18833   11255     -7578     
   + Partials       1604      26     -1578     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <ø> (-22.98%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.26% <ø> (-49.37%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [662 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...ad1d2d1](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 031b8fa2a947f69815bce9fa181dc98dd972d07e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [hudi] vinothchandar commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r519497597



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       What is holding the avro schema? this seems like a member held in the class you wrote? 




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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751) 
   * 1fa675d9739656c45850d60fa49b66970b18f5ac UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779",
       "triggerID" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 51ccc2db570a8cc996ff80725f668d7f6158aa24 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter commented on pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (01e47cc) into [master](https://codecov.io/gh/apache/hudi/commit/653981373339b8ce74b1227278d435f5c34d2deb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6539813) will **increase** coverage by `6.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   + Coverage     54.87%   61.85%   +6.97%     
   + Complexity     3836      340    -3496     
   ============================================
     Files           485       54     -431     
     Lines         23439     2016   -21423     
     Branches       2494      241    -2253     
   ============================================
   - Hits          12863     1247   -11616     
   + Misses         9423      646    -8777     
   + Partials       1153      123    -1030     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `61.85% <ø> (-7.79%)` | `340.00 <ø> (-39.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ies/exception/HoodieSnapshotExporterException.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2V4Y2VwdGlvbi9Ib29kaWVTbmFwc2hvdEV4cG9ydGVyRXhjZXB0aW9uLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../apache/hudi/utilities/HoodieSnapshotExporter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0hvb2RpZVNuYXBzaG90RXhwb3J0ZXIuamF2YQ==) | `5.17% <0.00%> (-83.63%)` | `0.00% <0.00%> (-28.00%)` | |
   | [...hudi/utilities/schema/JdbcbasedSchemaProvider.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9KZGJjYmFzZWRTY2hlbWFQcm92aWRlci5qYXZh) | `0.00% <0.00%> (-72.23%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...he/hudi/utilities/transform/AWSDmsTransformer.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3RyYW5zZm9ybS9BV1NEbXNUcmFuc2Zvcm1lci5qYXZh) | `0.00% <0.00%> (-66.67%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...in/java/org/apache/hudi/utilities/UtilHelpers.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1V0aWxIZWxwZXJzLmphdmE=) | `40.69% <0.00%> (-23.84%)` | `27.00% <0.00%> (-6.00%)` | |
   | [...c/main/java/org/apache/hudi/util/StreamerUtil.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS91dGlsL1N0cmVhbWVyVXRpbC5qYXZh) | | | |
   | [...i-cli/src/main/java/org/apache/hudi/cli/Table.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL1RhYmxlLmphdmE=) | | | |
   | [...he/hudi/exception/HoodieNotSupportedException.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZXhjZXB0aW9uL0hvb2RpZU5vdFN1cHBvcnRlZEV4Y2VwdGlvbi5qYXZh) | | | |
   | [...apache/hudi/common/engine/HoodieEngineContext.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2VuZ2luZS9Ib29kaWVFbmdpbmVDb250ZXh0LmphdmE=) | | | |
   | [...meline/versioning/clean/CleanMetadataMigrator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL3ZlcnNpb25pbmcvY2xlYW4vQ2xlYW5NZXRhZGF0YU1pZ3JhdG9yLmphdmE=) | | | |
   | ... and [424 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80ae3446670e4012ef3896477964bb916b71a864 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481






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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r519503664



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       yeah, no idea why the reduce fn is giving an issue. Casting to comprable works. But we might have to support user defined preCombine as well right. Also,  need to think about complex fields and not just simple fields.
   ```
   groupedDataset.reduceGroups((ReduceFunction<Row>) (v1, v2) ->
       {
         int compareVal = ((Comparable) v1.get(fieldIndex)).compareTo(v2.get(fieldIndex));
         if (compareVal >= 0) {
           return v1;
         } else {
           return v2;
         }
       });
   ```




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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5d68e03) into [master](https://codecov.io/gh/apache/hudi/commit/0d0dc6fb07e0c5496224c75052ab4f43d57b40f6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0d0dc6f) will **decrease** coverage by `45.86%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     55.14%   9.27%   -45.87%     
   + Complexity     3866      48     -3818     
   ============================================
     Files           488      54      -434     
     Lines         23619    2016    -21603     
     Branches       2528     241     -2287     
   ============================================
   - Hits          13024     187    -12837     
   + Misses         9437    1816     -7621     
   + Partials       1158      13     -1145     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `?` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `?` | |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.27% <ø> (-61.61%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [463 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664704566



##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();

Review comment:
       I took inspiration from existing code. 
   https://github.com/apache/hudi/blob/master/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java
   




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80ae3446670e4012ef3896477964bb916b71a864 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748) 
   * 0b5cdce9cea59681d7e604c6f6706a4e96a19861 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80ae3446670e4012ef3896477964bb916b71a864 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748) 
   * 0b5cdce9cea59681d7e604c6f6706a4e96a19861 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r511623934



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       @bvaradar : Is there some other option to go about deduping. Bcoz, in Bulksert with Rows, we don't have any HoodiePayload. Hence we have to operate on "Row"s only. So, have to group by keys and reduce by a function. In this patch, I have designed the function as this interface. 
   So, two questions.
   a. Is there a better option. 
   b. Even if we go with this option, I am getting task not serializable when executing this, since avro Schema is also sent along with Row. Also, wondering this might have any performance complications. 
   
   ```
   Caused by: java.io.NotSerializableException: org.apache.avro.Schema$RecordSchema
   Serialization stack:
   	- object not serializable (class: org.apache.avro.Schema$RecordSchema, value: {"type":"record","name":"trip","namespace":"example.schema","fields":[{"name":"_row_key","type":"string"},{"name":"partition","type":"string"},{"name":"ts","type":["long","null"]}]})
   	- field (class: org.apache.hudi.TestHoodieDatasetBulkInsertHelper, name: schema, type: class org.apache.avro.Schema)
   	- object (class org.apache.hudi.TestHoodieDatasetBulkInsertHelper, org.apache.hudi.TestHoodieDatasetBulkInsertHelper@8d7718e)
   	- field (class: org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow, name: this$0, type: class org.apache.hudi.TestHoodieDatasetBulkInsertHelper)
   	- object (class org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow, org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow@3436d3d7)
   	- element of array (index: 0)
   	- array (class [Ljava.lang.Object;, size 1)
   	- field (class: java.lang.invoke.SerializedLambda, name: capturedArgs, type: class [Ljava.lang.Object;)
   	- object (class java.lang.invoke.SerializedLambda, SerializedLambda[capturingClass=class org.apache.hudi.SparkRowWriteHelper, functionalInterfaceMethod=org/apache/spark/api/java/function/ReduceFunction.call:(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;, implementation=invokeStatic org/apache/hudi/SparkRowWriteHelper.lambda$deduplicateRows$14bf715c$1:(Lorg/apache/hudi/PreCombineRow;Lorg/apache/spark/sql/Row;Lorg/apache/spark/sql/Row;)Lorg/apache/spark/sql/Row;, instantiatedMethodType=(Lorg/apache/spark/sql/Row;Lorg/apache/spark/sql/Row;)Lorg/apache/spark/sql/Row;, numCaptured=1])
   	- writeReplace data (class: java.lang.invoke.SerializedLambda)
   	- object (class org.apache.hudi.SparkRowWriteHelper$$Lambda$306/2078785618, org.apache.hudi.SparkRowWriteHelper$$Lambda$306/2078785618@19d9ba89)
   	- field (class: org.apache.spark.sql.KeyValueGroupedDataset$$anonfun$reduceGroups$1, name: f$4, type: interface org.apache.spark.api.java.function.ReduceFunction)
   	- object (class org.apache.spark.sql.KeyValueGroupedDataset$$anonfun$reduceGroups$1, <function2>)
   	- field (class: org.apache.spark.sql.expressions.ReduceAggregator, name: func, type: interface scala.Function2)
   	- object (class org.apache.spark.sql.expressions.ReduceAggregator, org.apache.spark.sql.expressions.ReduceAggregator@14af73e1)
   ```
   




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

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



[GitHub] [hudi] nsivabalan merged pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan merged pull request #2206:
URL: https://github.com/apache/hudi/pull/2206


   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (39b3315) into [master](https://codecov.io/gh/apache/hudi/commit/653981373339b8ce74b1227278d435f5c34d2deb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6539813) will **increase** coverage by `14.71%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2206       +/-   ##
   =============================================
   + Coverage     54.87%   69.59%   +14.71%     
   + Complexity     3836      378     -3458     
   =============================================
     Files           485       54      -431     
     Lines         23439     2016    -21423     
     Branches       2494      241     -2253     
   =============================================
   - Hits          12863     1403    -11460     
   + Misses         9423      479     -8944     
   + Partials       1153      134     -1019     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.59% <ø> (-0.05%)` | `378.00 <ø> (-1.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `70.84% <0.00%> (-0.34%)` | `55.00% <0.00%> (-1.00%)` | |
   | [...i-cli/src/main/java/org/apache/hudi/cli/Table.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL1RhYmxlLmphdmE=) | | | |
   | [...java/org/apache/hudi/common/util/CleanerUtils.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvQ2xlYW5lclV0aWxzLmphdmE=) | | | |
   | [...main/java/org/apache/hudi/dla/HoodieDLAClient.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktZGxhLXN5bmMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZGxhL0hvb2RpZURMQUNsaWVudC5qYXZh) | | | |
   | [...va/org/apache/hudi/common/fs/ConsistencyGuard.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL0NvbnNpc3RlbmN5R3VhcmQuamF2YQ==) | | | |
   | [...n/java/org/apache/hudi/internal/DefaultSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmsyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2ludGVybmFsL0RlZmF1bHRTb3VyY2UuamF2YQ==) | | | |
   | [...di/timeline/service/handlers/FileSliceHandler.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS10aW1lbGluZS1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3RpbWVsaW5lL3NlcnZpY2UvaGFuZGxlcnMvRmlsZVNsaWNlSGFuZGxlci5qYXZh) | | | |
   | [.../org/apache/hudi/sink/utils/NonThrownExecutor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9zaW5rL3V0aWxzL05vblRocm93bkV4ZWN1dG9yLmphdmE=) | | | |
   | [...mmon/table/log/AbstractHoodieLogRecordScanner.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9BYnN0cmFjdEhvb2RpZUxvZ1JlY29yZFNjYW5uZXIuamF2YQ==) | | | |
   | [...rg/apache/hudi/cli/commands/SavepointsCommand.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1NhdmVwb2ludHNDb21tYW5kLmphdmE=) | | | |
   | ... and [420 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664884833



##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(
+        (MapFunction<Row, String>) value ->
+            isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
+                (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
+        .reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
+          if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
+            return v1;
+          } else {
+            return v2;
+          }
+            }
+        ).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
+  }
+
+  private ExpressionEncoder getEncoder(StructType schema) {
+    List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
+        .map(Attribute::toAttribute).collect(Collectors.toList());
+    return RowEncoder.apply(schema)

Review comment:
       yes, tested both spark2 and spark3. 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (51ccc2d) into [master](https://codecov.io/gh/apache/hudi/commit/ea9e5d0e8b7557ef82631ac173d67f15bad13690?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ea9e5d0) will **decrease** coverage by `31.98%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2206       +/-   ##
   =============================================
   - Coverage     47.61%   15.63%   -31.99%     
   + Complexity     5495      487     -5008     
   =============================================
     Files           929      281      -648     
     Lines         41240    11618    -29622     
     Branches       4135      950     -3185     
   =============================================
   - Hits          19637     1816    -17821     
   + Misses        19859     9643    -10216     
   + Partials       1744      159     -1585     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <ø> (-34.61%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.62% <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/hudi/metrics/MetricsReporter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/metrics/MetricsReporterType.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/client/bootstrap/BootstrapMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC9ib290c3RyYXAvQm9vdHN0cmFwTW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC91dGlscy9Db25jYXRlbmF0aW5nSXRlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/hudi/config/HoodieMetricsPrometheusConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NvbmZpZy9Ib29kaWVNZXRyaWNzUHJvbWV0aGV1c0NvbmZpZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2V4ZWN1dGlvbi9idWxraW5zZXJ0L0J1bGtJbnNlcnRTb3J0TW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...able/action/compact/CompactionTriggerStrategy.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3RhYmxlL2FjdGlvbi9jb21wYWN0L0NvbXBhY3Rpb25UcmlnZ2VyU3RyYXRlZ3kuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [713 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [ea9e5d0...51ccc2d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664705394



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -333,7 +334,9 @@ private[hudi] object HoodieSparkSqlWriter {
     log.info(s"Registered avro schema : ${schema.toString(true)}")
     val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
     val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
+    val isGlobalIndex = SparkHoodieIndex.createIndex(writeConfig).isGlobal

Review comment:
       may be for out of the box index types, we can maintain a static map of IndexType -> isGlobal and use it. wdyt? 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-736689026


   @bvaradar @vinothchandar : patch is ready for review


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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fa675d9739656c45850d60fa49b66970b18f5ac Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757) 
   * 51ccc2db570a8cc996ff80725f668d7f6158aa24 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r511623934



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       @bvaradar : Is there some other option to go about deduping multiple Rows. Bcoz, in Bulksert with Rows, we don't have any HoodiePayload. Hence we have to operate on "Row"s only. So, have to group by keys and reduce by a function. In this patch, I have designed the function as this interface. 
   So, two questions.
   a. Is there a better option. 
   b. Even if we go with this option, I am getting task not serializable when executing this, since avro Schema is also sent along with Row. Also, wondering this might have any performance complications. 
   
   ```
   Caused by: java.io.NotSerializableException: org.apache.avro.Schema$RecordSchema
   Serialization stack:
   	- object not serializable (class: org.apache.avro.Schema$RecordSchema, value: {"type":"record","name":"trip","namespace":"example.schema","fields":[{"name":"_row_key","type":"string"},{"name":"partition","type":"string"},{"name":"ts","type":["long","null"]}]})
   	- field (class: org.apache.hudi.TestHoodieDatasetBulkInsertHelper, name: schema, type: class org.apache.avro.Schema)
   	- object (class org.apache.hudi.TestHoodieDatasetBulkInsertHelper, org.apache.hudi.TestHoodieDatasetBulkInsertHelper@8d7718e)
   	- field (class: org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow, name: this$0, type: class org.apache.hudi.TestHoodieDatasetBulkInsertHelper)
   	- object (class org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow, org.apache.hudi.TestHoodieDatasetBulkInsertHelper$TestPreCombineRow@3436d3d7)
   	- element of array (index: 0)
   	- array (class [Ljava.lang.Object;, size 1)
   	- field (class: java.lang.invoke.SerializedLambda, name: capturedArgs, type: class [Ljava.lang.Object;)
   	- object (class java.lang.invoke.SerializedLambda, SerializedLambda[capturingClass=class org.apache.hudi.SparkRowWriteHelper, functionalInterfaceMethod=org/apache/spark/api/java/function/ReduceFunction.call:(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;, implementation=invokeStatic org/apache/hudi/SparkRowWriteHelper.lambda$deduplicateRows$14bf715c$1:(Lorg/apache/hudi/PreCombineRow;Lorg/apache/spark/sql/Row;Lorg/apache/spark/sql/Row;)Lorg/apache/spark/sql/Row;, instantiatedMethodType=(Lorg/apache/spark/sql/Row;Lorg/apache/spark/sql/Row;)Lorg/apache/spark/sql/Row;, numCaptured=1])
   	- writeReplace data (class: java.lang.invoke.SerializedLambda)
   	- object (class org.apache.hudi.SparkRowWriteHelper$$Lambda$306/2078785618, org.apache.hudi.SparkRowWriteHelper$$Lambda$306/2078785618@19d9ba89)
   	- field (class: org.apache.spark.sql.KeyValueGroupedDataset$$anonfun$reduceGroups$1, name: f$4, type: interface org.apache.spark.api.java.function.ReduceFunction)
   	- object (class org.apache.spark.sql.KeyValueGroupedDataset$$anonfun$reduceGroups$1, <function2>)
   	- field (class: org.apache.spark.sql.expressions.ReduceAggregator, name: func, type: interface scala.Function2)
   	- object (class org.apache.spark.sql.expressions.ReduceAggregator, org.apache.spark.sql.expressions.ReduceAggregator@14af73e1)
   ```
   




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

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481






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

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1fa675d) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **decrease** coverage by `4.20%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   - Coverage     45.85%   41.64%   -4.21%     
   + Complexity     4699     4052     -647     
   ============================================
     Files           828      750      -78     
     Lines         37742    33346    -4396     
     Branches       3780     3100     -680     
   ============================================
   - Hits          17305    13886    -3419     
   + Misses        18833    18351     -482     
   + Partials       1604     1109     -495     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `39.97% <ø> (ø)` | |
   | hudiclient | `22.97% <ø> (ø)` | |
   | hudicommon | `48.56% <ø> (ø)` | |
   | hudiflink | `59.58% <ø> (ø)` | |
   | hudihadoopmr | `51.29% <ø> (ø)` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.59% <ø> (-0.04%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/hive/HiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNDb25maWcuamF2YQ==) | `0.00% <0.00%> (-98.08%)` | :arrow_down: |
   | [...he/hudi/hive/replication/GlobalHiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvcmVwbGljYXRpb24vR2xvYmFsSGl2ZVN5bmNDb25maWcuamF2YQ==) | `0.00% <0.00%> (-95.00%)` | :arrow_down: |
   | [.../apache/hudi/hive/MultiPartKeysValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTXVsdGlQYXJ0S2V5c1ZhbHVlRXh0cmFjdG9yLmphdmE=) | `0.00% <0.00%> (-90.91%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/SchemaDifference.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvU2NoZW1hRGlmZmVyZW5jZS5qYXZh) | `0.00% <0.00%> (-84.85%)` | :arrow_down: |
   | [...he/hudi/hive/replication/ReplicationStateSync.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvcmVwbGljYXRpb24vUmVwbGljYXRpb25TdGF0ZVN5bmMuamF2YQ==) | `0.00% <0.00%> (-80.77%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/util/ConfigUtils.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db25maWdVdGlscy5qYXZh) | `0.00% <0.00%> (-73.92%)` | :arrow_down: |
   | [...c/main/java/org/apache/hudi/hive/HiveSyncTool.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNUb29sLmphdmE=) | `0.00% <0.00%> (-72.36%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/HoodieHiveClient.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSG9vZGllSGl2ZUNsaWVudC5qYXZh) | `0.00% <0.00%> (-70.14%)` | :arrow_down: |
   | ... and [85 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...1fa675d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1fa675d) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **decrease** coverage by `42.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     45.85%   2.87%   -42.98%     
   + Complexity     4699      82     -4617     
   ============================================
     Files           828     280      -548     
     Lines         37742   11615    -26127     
     Branches       3780     950     -2830     
   ============================================
   - Hits          17305     334    -16971     
   + Misses        18833   11255     -7578     
   + Partials       1604      26     -1578     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <ø> (-22.98%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.26% <ø> (-49.37%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [662 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...1fa675d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (031b8fa) into [master](https://codecov.io/gh/apache/hudi/commit/0d0dc6fb07e0c5496224c75052ab4f43d57b40f6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0d0dc6f) will **decrease** coverage by `2.62%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   - Coverage     55.14%   52.51%   -2.63%     
   + Complexity     3866      404    -3462     
   ============================================
     Files           488       69     -419     
     Lines         23619     2866   -20753     
     Branches       2528      358    -2170     
   ============================================
   - Hits          13024     1505   -11519     
   + Misses         9437     1218    -8219     
   + Partials       1158      143    -1015     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `?` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `6.77% <ø> (-39.83%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `71.06% <ø> (+0.18%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/hive/HiveSyncConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNDb25maWcuamF2YQ==) | `0.00% <0.00%> (-97.83%)` | :arrow_down: |
   | [.../apache/hudi/hive/MultiPartKeysValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTXVsdGlQYXJ0S2V5c1ZhbHVlRXh0cmFjdG9yLmphdmE=) | `0.00% <0.00%> (-90.91%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/SchemaDifference.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvU2NoZW1hRGlmZmVyZW5jZS5qYXZh) | `0.00% <0.00%> (-84.85%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/util/ConfigUtils.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db25maWdVdGlscy5qYXZh) | `0.00% <0.00%> (-73.92%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/hive/HoodieHiveClient.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSG9vZGllSGl2ZUNsaWVudC5qYXZh) | `0.00% <0.00%> (-70.33%)` | :arrow_down: |
   | [...va/org/apache/hudi/hive/util/ColumnNameXLator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9Db2x1bW5OYW1lWExhdG9yLmphdmE=) | `0.00% <0.00%> (-70.00%)` | :arrow_down: |
   | [...java/org/apache/hudi/hive/util/HiveSchemaUtil.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvdXRpbC9IaXZlU2NoZW1hVXRpbC5qYXZh) | `0.00% <0.00%> (-68.94%)` | :arrow_down: |
   | [...c/main/java/org/apache/hudi/hive/HiveSyncTool.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN5bmNUb29sLmphdmE=) | `0.00% <0.00%> (-67.55%)` | :arrow_down: |
   | ... and [424 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779",
       "triggerID" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=781",
       "triggerID" : "875798888",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 51ccc2db570a8cc996ff80725f668d7f6158aa24 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=781) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r519501622



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       let me investigate this. might be in my dedup logic. 
   ```
   public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, PreCombineRow preCombineRow) {
       ExpressionEncoder encoder = getEncoder(inputDf.schema());
       return inputDf.groupByKey(
           (MapFunction<Row, String>) value -> value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), Encoders.STRING())
           .reduceGroups((ReduceFunction<Row>) (v1, v2) -> preCombineRow.combineTwoRows(v1, v2)).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
     }
   ```
   




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664705394



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -333,7 +334,9 @@ private[hudi] object HoodieSparkSqlWriter {
     log.info(s"Registered avro schema : ${schema.toString(true)}")
     val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
     val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
+    val isGlobalIndex = SparkHoodieIndex.createIndex(writeConfig).isGlobal

Review comment:
       may be for out of the box index types, we can maintain a static map of IndexType to isGlobal and use it. wdyt? 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1fa675d) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **decrease** coverage by `30.22%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2206       +/-   ##
   =============================================
   - Coverage     45.85%   15.62%   -30.23%     
   + Complexity     4699      486     -4213     
   =============================================
     Files           828      280      -548     
     Lines         37742    11615    -26127     
     Branches       3780      950     -2830     
   =============================================
   - Hits          17305     1815    -15490     
   + Misses        18833     9640     -9193     
   + Partials       1604      160     -1444     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <ø> (-22.98%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.59% <ø> (-0.04%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/hudi/metrics/MetricsReporter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/metrics/MetricsReporterType.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvTWV0cmljc1JlcG9ydGVyVHlwZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/client/bootstrap/BootstrapMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC9ib290c3RyYXAvQm9vdHN0cmFwTW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/hudi/hive/HiveStylePartitionValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvSGl2ZVN0eWxlUGFydGl0aW9uVmFsdWVFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NsaWVudC91dGlscy9Db25jYXRlbmF0aW5nSXRlcmF0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...che/hudi/config/HoodieMetricsPrometheusConfig.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2NvbmZpZy9Ib29kaWVNZXRyaWNzUHJvbWV0aGV1c0NvbmZpZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2V4ZWN1dGlvbi9idWxraW5zZXJ0L0J1bGtJbnNlcnRTb3J0TW9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...able/action/compact/CompactionTriggerStrategy.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3RhYmxlL2FjdGlvbi9jb21wYWN0L0NvbXBhY3Rpb25UcmlnZ2VyU3RyYXRlZ3kuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [615 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...1fa675d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664708643



##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(
+        (MapFunction<Row, String>) value ->
+            isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
+                (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
+        .reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
+          if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
+            return v1;
+          } else {
+            return v2;
+          }
+            }
+        ).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
+  }
+
+  private ExpressionEncoder getEncoder(StructType schema) {
+    List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
+        .map(Attribute::toAttribute).collect(Collectors.toList());
+    return RowEncoder.apply(schema)

Review comment:
       its been quite sometime I put up this patch :). Will do a round of testing and will update for both spark versions. 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] vinothchandar commented on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861665899


   Lets give this a shot. 
   
   @hudi-bot run azure


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

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c351cff) into [master](https://codecov.io/gh/apache/hudi/commit/653981373339b8ce74b1227278d435f5c34d2deb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6539813) will **increase** coverage by `14.71%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2206       +/-   ##
   =============================================
   + Coverage     54.87%   69.59%   +14.71%     
   + Complexity     3836      378     -3458     
   =============================================
     Files           485       54      -431     
     Lines         23439     2016    -21423     
     Branches       2494      241     -2253     
   =============================================
   - Hits          12863     1403    -11460     
   + Misses         9423      479     -8944     
   + Partials       1153      134     -1019     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.59% <ø> (-0.05%)` | `378.00 <ø> (-1.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `70.84% <0.00%> (-0.34%)` | `55.00% <0.00%> (-1.00%)` | |
   | [.../apache/hudi/common/model/HoodieRecordPayload.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZVJlY29yZFBheWxvYWQuamF2YQ==) | | | |
   | [...e/hudi/cli/commands/UpgradeOrDowngradeCommand.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1VwZ3JhZGVPckRvd25ncmFkZUNvbW1hbmQuamF2YQ==) | | | |
   | [...udi/common/util/queue/BoundedInMemoryExecutor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvcXVldWUvQm91bmRlZEluTWVtb3J5RXhlY3V0b3IuamF2YQ==) | | | |
   | [...src/main/java/org/apache/hudi/dla/DLASyncTool.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktZGxhLXN5bmMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZGxhL0RMQVN5bmNUb29sLmphdmE=) | | | |
   | [.../hudi/common/model/HoodieRecordGlobalLocation.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZVJlY29yZEdsb2JhbExvY2F0aW9uLmphdmE=) | | | |
   | [...e/hudi/table/format/mor/MergeOnReadInputSplit.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS90YWJsZS9mb3JtYXQvbW9yL01lcmdlT25SZWFkSW5wdXRTcGxpdC5qYXZh) | | | |
   | [...e/hudi/common/fs/SchemeAwareFSDataInputStream.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL1NjaGVtZUF3YXJlRlNEYXRhSW5wdXRTdHJlYW0uamF2YQ==) | | | |
   | [...i/hadoop/utils/HoodieRealtimeInputFormatUtils.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3V0aWxzL0hvb2RpZVJlYWx0aW1lSW5wdXRGb3JtYXRVdGlscy5qYXZh) | | | |
   | [...i/common/table/view/FileSystemViewStorageType.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3ZpZXcvRmlsZVN5c3RlbVZpZXdTdG9yYWdlVHlwZS5qYXZh) | | | |
   | ... and [420 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] vinothchandar commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664284205



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -98,6 +98,8 @@
   public static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
   public static final String COMBINE_BEFORE_DELETE_PROP = "hoodie.combine.before.delete";
   public static final String DEFAULT_COMBINE_BEFORE_DELETE = "true";
+  public static final String COMBINE_BEFORE_BULK_INSERT_PROP = "hoodie.combine.before.bulk.insert";

Review comment:
       all these configs need to be redone based ConfigProperty/HoodieConfig

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();

Review comment:
       why the singleton etc? Can't we just use a static method?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+

Review comment:
       nit: extra line?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
##########
@@ -96,9 +97,15 @@
                 functions.lit("").cast(DataTypes.StringType))
             .withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
                 functions.lit("").cast(DataTypes.StringType));
+
+    Dataset<Row> dedupedDf = rowDatasetWithHoodieColumns;
+    if (config.shouldCombineBeforeBulkInsert()) {

Review comment:
       I understand that the new config is just used here as of this PR. but from an user standpoint, on the non-row writer path, combine.before.insert was controlling this already. We should just make it consistent. 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -306,6 +308,10 @@ public boolean shouldCombineBeforeInsert() {
     return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
   }
 
+  public boolean shouldCombineBeforeBulkInsert() {

Review comment:
       So far, we have used one config `combine.before.insert` to control it for both insert and bulk_insert. Can we keep it the same way? Otherwise, wont it be backwards incompatible, ie a user can be expecting the combine.before.insert continue to take effect for bulk_insert as well and it won't be the case?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(
+        (MapFunction<Row, String>) value ->
+            isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
+                (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
+        .reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
+          if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
+            return v1;
+          } else {
+            return v2;
+          }
+            }
+        ).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
+  }
+
+  private ExpressionEncoder getEncoder(StructType schema) {
+    List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
+        .map(Attribute::toAttribute).collect(Collectors.toList());
+    return RowEncoder.apply(schema)

Review comment:
       have you tested with both Spark 2 and 3?  Some of these classes can be different and actually fail?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(

Review comment:
       lets use reduceByKey(), which we use for RDD path? groupByKey() can hog memory. 

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -333,7 +334,9 @@ private[hudi] object HoodieSparkSqlWriter {
     log.info(s"Registered avro schema : ${schema.toString(true)}")
     val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
     val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
-    val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
+    val isGlobalIndex = SparkHoodieIndex.createIndex(writeConfig).isGlobal

Review comment:
       do we need to create the index to really check if its Global? Wondering if there are simpler means. (may be not, given we support even user defined indexes) 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1fa675d) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **increase** coverage by `0.01%`.
   > The diff coverage is `86.36%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   + Coverage     45.85%   45.86%   +0.01%     
   - Complexity     4699     4705       +6     
   ============================================
     Files           828      829       +1     
     Lines         37742    37762      +20     
     Branches       3780     3781       +1     
   ============================================
   + Hits          17305    17321      +16     
   - Misses        18833    18834       +1     
   - Partials       1604     1607       +3     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `39.97% <ø> (ø)` | |
   | hudiclient | `22.97% <ø> (ø)` | |
   | hudicommon | `48.56% <ø> (ø)` | |
   | hudiflink | `59.58% <ø> (ø)` | |
   | hudihadoopmr | `51.29% <ø> (ø)` | |
   | hudisparkdatasource | `67.42% <86.36%> (+0.09%)` | :arrow_up: |
   | hudisync | `54.48% <ø> (ø)` | |
   | huditimelineservice | `64.07% <ø> (ø)` | |
   | hudiutilities | `58.59% <ø> (-0.04%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...main/java/org/apache/hudi/SparkRowWriteHelper.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvU3BhcmtSb3dXcml0ZUhlbHBlci5qYXZh) | `80.00% <80.00%> (ø)` | |
   | [...org/apache/hudi/HoodieDatasetBulkInsertHelper.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllRGF0YXNldEJ1bGtJbnNlcnRIZWxwZXIuamF2YQ==) | `97.22% <100.00%> (+0.25%)` | :arrow_up: |
   | [...n/scala/org/apache/hudi/HoodieSparkSqlWriter.scala](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL0hvb2RpZVNwYXJrU3FsV3JpdGVyLnNjYWxh) | `71.34% <100.00%> (+0.16%)` | :arrow_up: |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.14% <0.00%> (-0.34%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...1fa675d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r519503664



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       yeah, no idea why the reduce fn is giving an issue. Casting to comprable works, but this may not be good enough right. We need to think about complex fields and not just simple fields. Also, we might have to support user defined preCombine. 
   
   ```
   groupedDataset.reduceGroups((ReduceFunction<Row>) (v1, v2) ->
       {
         int compareVal = ((Comparable) v1.get(fieldIndex)).compareTo(v2.get(fieldIndex));
         if (compareVal >= 0) {
           return v1;
         } else {
           return v2;
         }
       });
   ```




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

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



[GitHub] [hudi] vinothchandar commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r520267589



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       @nsivabalan could we take the same approach as key generators, extend `HoodieRecordPayload` with new methods for specifying how two rows should combine, but still honor the avro based impl, by converting row -> avro as 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.

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r519501024



##########
File path: hudi-spark/src/main/java/org/apache/hudi/PreCombineRow.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hudi;
+
+import org.apache.spark.sql.Row;
+
+import java.io.Serializable;
+
+/**
+ * Interface used to preCombine two Spark sql Rows.
+ */
+public interface PreCombineRow extends Serializable {

Review comment:
       nope. Thats is what is confusing me. 
   ```
     class TestPreCombineRow implements PreCombineRow {
   
       @Override
       public Row combineTwoRows(Row v1, Row v2) {
         long tsV1 = v1.getAs("ts");
         long tsV2 = v2.getAs("ts");
         return (tsV1 >= tsV2) ? v1 : v2;
       }
     }
   ```
   




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

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (80ae344) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **decrease** coverage by `42.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     45.85%   2.87%   -42.98%     
   + Complexity     4699      82     -4617     
   ============================================
     Files           828     280      -548     
     Lines         37742   11615    -26127     
     Branches       3780     950     -2830     
   ============================================
   - Hits          17305     334    -16971     
   + Misses        18833   11255     -7578     
   + Partials       1604      26     -1578     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <ø> (-22.98%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.28% <ø> (-49.20%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.26% <ø> (-49.37%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldHJpY3MvSG9vZGllR2F1Z2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [662 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...80ae344](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1fa675d) into [master](https://codecov.io/gh/apache/hudi/commit/221ddd9bf3899e3672210404d51e686770ba446d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (221ddd9) will **increase** coverage by `1.77%`.
   > The diff coverage is `63.33%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   + Coverage     45.85%   47.62%   +1.77%     
   - Complexity     4699     5494     +795     
   ============================================
     Files           828      925      +97     
     Lines         37742    41234    +3492     
     Branches       3780     4134     +354     
   ============================================
   + Hits          17305    19636    +2331     
   - Misses        18833    19853    +1020     
   - Partials       1604     1745     +141     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `39.97% <ø> (ø)` | |
   | hudiclient | `34.56% <0.00%> (+11.58%)` | :arrow_up: |
   | hudicommon | `48.56% <ø> (ø)` | |
   | hudiflink | `59.58% <ø> (ø)` | |
   | hudihadoopmr | `51.29% <ø> (ø)` | |
   | hudisparkdatasource | `67.42% <86.36%> (+0.09%)` | :arrow_up: |
   | hudisync | `54.48% <ø> (ø)` | |
   | huditimelineservice | `64.07% <ø> (ø)` | |
   | hudiutilities | `58.59% <ø> (-0.04%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/java/org/apache/hudi/index/SparkHoodieIndex.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW5kZXgvU3BhcmtIb29kaWVJbmRleC5qYXZh) | `56.52% <0.00%> (ø)` | |
   | [...main/java/org/apache/hudi/SparkRowWriteHelper.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvU3BhcmtSb3dXcml0ZUhlbHBlci5qYXZh) | `80.00% <80.00%> (ø)` | |
   | [...org/apache/hudi/HoodieDatasetBulkInsertHelper.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllRGF0YXNldEJ1bGtJbnNlcnRIZWxwZXIuamF2YQ==) | `97.22% <100.00%> (+0.25%)` | :arrow_up: |
   | [...n/scala/org/apache/hudi/HoodieSparkSqlWriter.scala](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL0hvb2RpZVNwYXJrU3FsV3JpdGVyLnNjYWxh) | `71.34% <100.00%> (+0.16%)` | :arrow_up: |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.14% <0.00%> (-0.34%)` | :arrow_down: |
   | [...er/SparkExecuteClusteringCommitActionExecutor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdGFibGUvYWN0aW9uL2NsdXN0ZXIvU3BhcmtFeGVjdXRlQ2x1c3RlcmluZ0NvbW1pdEFjdGlvbkV4ZWN1dG9yLmphdmE=) | `89.42% <0.00%> (ø)` | |
   | [...java/org/apache/hudi/client/HoodieWriteResult.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpZW50L0hvb2RpZVdyaXRlUmVzdWx0LmphdmE=) | `46.15% <0.00%> (ø)` | |
   | [...e/hudi/client/common/HoodieSparkEngineContext.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpZW50L2NvbW1vbi9Ib29kaWVTcGFya0VuZ2luZUNvbnRleHQuamF2YQ==) | `54.83% <0.00%> (ø)` | |
   | [.../org/apache/hudi/keygen/RowKeyGeneratorHelper.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkva2V5Z2VuL1Jvd0tleUdlbmVyYXRvckhlbHBlci5qYXZh) | `67.44% <0.00%> (ø)` | |
   | [...apache/hudi/keygen/TimestampBasedKeyGenerator.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jbGllbnQvaHVkaS1zcGFyay1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkva2V5Z2VuL1RpbWVzdGFtcEJhc2VkS2V5R2VuZXJhdG9yLmphdmE=) | `80.95% <0.00%> (ø)` | |
   | ... and [92 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [221ddd9...1fa675d](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (63ca76b) into [master](https://codecov.io/gh/apache/hudi/commit/653981373339b8ce74b1227278d435f5c34d2deb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6539813) will **decrease** coverage by `45.60%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 63ca76b differs from pull request most recent head 39b3315. Consider uploading reports for the commit 39b3315 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     54.87%   9.27%   -45.61%     
   + Complexity     3836      48     -3788     
   ============================================
     Files           485      54      -431     
     Lines         23439    2016    -21423     
     Branches       2494     241     -2253     
   ============================================
   - Hits          12863     187    -12676     
   + Misses         9423    1816     -7607     
   + Partials       1153      13     -1140     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.27% <ø> (-60.37%)` | `48.00 <ø> (-331.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [460 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664696137



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -98,6 +98,8 @@
   public static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
   public static final String COMBINE_BEFORE_DELETE_PROP = "hoodie.combine.before.delete";
   public static final String DEFAULT_COMBINE_BEFORE_DELETE = "true";
+  public static final String COMBINE_BEFORE_BULK_INSERT_PROP = "hoodie.combine.before.bulk.insert";

Review comment:
       yes.




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779",
       "triggerID" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=781",
       "triggerID" : "875798888",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 51ccc2db570a8cc996ff80725f668d7f6158aa24 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=781) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-875798888


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fa675d9739656c45850d60fa49b66970b18f5ac Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (10043f2) into [master](https://codecov.io/gh/apache/hudi/commit/0d0dc6fb07e0c5496224c75052ab4f43d57b40f6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0d0dc6f) will **increase** coverage by `7.95%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2206      +/-   ##
   ============================================
   + Coverage     55.14%   63.09%   +7.95%     
   + Complexity     3866      347    -3519     
   ============================================
     Files           488       54     -434     
     Lines         23619     2016   -21603     
     Branches       2528      241    -2287     
   ============================================
   - Hits          13024     1272   -11752     
   + Misses         9437      621    -8816     
   + Partials       1158      123    -1035     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `?` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `?` | |
   | huditimelineservice | `?` | |
   | hudiutilities | `63.09% <ø> (-7.79%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ies/exception/HoodieSnapshotExporterException.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2V4Y2VwdGlvbi9Ib29kaWVTbmFwc2hvdEV4cG9ydGVyRXhjZXB0aW9uLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/hudi/utilities/HoodieSnapshotExporter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0hvb2RpZVNuYXBzaG90RXhwb3J0ZXIuamF2YQ==) | `5.17% <0.00%> (-83.63%)` | :arrow_down: |
   | [...hudi/utilities/schema/JdbcbasedSchemaProvider.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9KZGJjYmFzZWRTY2hlbWFQcm92aWRlci5qYXZh) | `0.00% <0.00%> (-72.23%)` | :arrow_down: |
   | [...he/hudi/utilities/transform/AWSDmsTransformer.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3RyYW5zZm9ybS9BV1NEbXNUcmFuc2Zvcm1lci5qYXZh) | `0.00% <0.00%> (-66.67%)` | :arrow_down: |
   | [...in/java/org/apache/hudi/utilities/UtilHelpers.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1V0aWxIZWxwZXJzLmphdmE=) | `40.69% <0.00%> (-23.84%)` | :arrow_down: |
   | [.../hudi/common/model/DefaultHoodieRecordPayload.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0RlZmF1bHRIb29kaWVSZWNvcmRQYXlsb2FkLmphdmE=) | | |
   | [...che/hudi/common/table/timeline/HoodieTimeline.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZVRpbWVsaW5lLmphdmE=) | | |
   | [...spark/src/main/scala/org/apache/hudi/package.scala](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL3BhY2thZ2Uuc2NhbGE=) | | |
   | [...a/org/apache/hudi/common/bloom/InternalFilter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2Jsb29tL0ludGVybmFsRmlsdGVyLmphdmE=) | | |
   | [...che/hudi/metadata/TimelineMergedTableMetadata.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvbWV0YWRhdGEvVGltZWxpbmVNZXJnZWRUYWJsZU1ldGFkYXRhLmphdmE=) | | |
   | ... and [427 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0b5cdce9cea59681d7e604c6f6706a4e96a19861 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749) 
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 031b8fa2a947f69815bce9fa181dc98dd972d07e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664696403



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -306,6 +308,10 @@ public boolean shouldCombineBeforeInsert() {
     return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
   }
 
+  public boolean shouldCombineBeforeBulkInsert() {

Review comment:
       yeah, makes sense to use combine.before.insert only. will remove the new config. 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 031b8fa2a947f69815bce9fa181dc98dd972d07e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202) 
   * 80ae3446670e4012ef3896477964bb916b71a864 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#discussion_r664708105



##########
File path: hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/SparkRowWriteHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.model.HoodieRecord;
+
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.ReduceFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.Attribute;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.JavaConverters;
+
+/**
+ * Helper class to assist in deduplicating Rows for BulkInsert with Rows.
+ */
+public class SparkRowWriteHelper {
+
+  private SparkRowWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+
+    private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
+  }
+
+  public static SparkRowWriteHelper newInstance() {
+    return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
+    ExpressionEncoder encoder = getEncoder(inputDf.schema());
+
+    return inputDf.groupByKey(

Review comment:
       I too badly wanted to use one, but unfortunately there is none :( 
   https://stackoverflow.com/questions/38383207/rolling-your-own-reducebykey-in-spark-dataset
   https://stackoverflow.com/questions/57359260/why-there-is-no-reducebykey-in-sparks-dataset
   hence, had to go with groupByKey and then do reduceBy. 




-- 
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: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751) 
   * 1fa675d9739656c45850d60fa49b66970b18f5ac Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot edited a comment on pull request #2206: [HUDI-1105] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-861663727


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=201",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "861665899",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=202",
       "triggerID" : "031b8fa2a947f69815bce9fa181dc98dd972d07e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80ae3446670e4012ef3896477964bb916b71a864",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=748",
       "triggerID" : "80ae3446670e4012ef3896477964bb916b71a864",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=749",
       "triggerID" : "0b5cdce9cea59681d7e604c6f6706a4e96a19861",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=751",
       "triggerID" : "ad1d2d1a2f38cfb396d8f3d30f2f9572a1506e90",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757",
       "triggerID" : "1fa675d9739656c45850d60fa49b66970b18f5ac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779",
       "triggerID" : "51ccc2db570a8cc996ff80725f668d7f6158aa24",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1fa675d9739656c45850d60fa49b66970b18f5ac Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=757) 
   * 51ccc2db570a8cc996ff80725f668d7f6158aa24 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=779) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (031b8fa) into [master](https://codecov.io/gh/apache/hudi/commit/0d0dc6fb07e0c5496224c75052ab4f43d57b40f6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0d0dc6f) will **decrease** coverage by `46.66%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     55.14%   8.47%   -46.67%     
   + Complexity     3866      62     -3804     
   ============================================
     Files           488      69      -419     
     Lines         23619    2866    -20753     
     Branches       2528     358     -2170     
   ============================================
   - Hits          13024     243    -12781     
   + Misses         9437    2602     -6835     
   + Partials       1158      21     -1137     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `?` | |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `6.77% <ø> (-39.83%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.17% <ø> (-61.72%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [464 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #2206: [WIP] Adding dedup support for Bulk Insert w/ Rows

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2206:
URL: https://github.com/apache/hudi/pull/2206#issuecomment-846504481


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2206](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c351cff) into [master](https://codecov.io/gh/apache/hudi/commit/653981373339b8ce74b1227278d435f5c34d2deb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6539813) will **decrease** coverage by `45.60%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2206/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2206       +/-   ##
   ============================================
   - Coverage     54.87%   9.27%   -45.61%     
   + Complexity     3836      48     -3788     
   ============================================
     Files           485      54      -431     
     Lines         23439    2016    -21423     
     Branches       2494     241     -2253     
   ============================================
   - Hits          12863     187    -12676     
   + Misses         9423    1816     -7607     
   + Partials       1153      13     -1140     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.27% <ø> (-60.37%)` | `48.00 <ø> (-331.00)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2206?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [460 more](https://codecov.io/gh/apache/hudi/pull/2206/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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

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