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 2021/06/29 08:14:32 UTC

[GitHub] [hudi] xiarixiaoyao opened a new pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

xiarixiaoyao opened a new pull request #3182:
URL: https://github.com/apache/hudi/pull/3182


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   fix the bug that metatable cannot support non_partition table
   
   now, we found that when we enable metable for non_partition hudi table,  the follow  error occur:
   org.apache.hudi.exception.HoodieMetadataException: Error syncing to metadata table.org.apache.hudi.exception.HoodieMetadataException: Error syncing to metadata table.
   at org.apache.hudi.client.SparkRDDWriteClient.syncTableMetadata(SparkRDDWriteClient.java:447) at org.apache.hudi.client.AbstractHoodieWriteClient.postCommit(AbstractHoodieWriteClient.java:433) at org.apache.hudi.client.AbstractHoodieWriteClient.commitStats(AbstractHoodieWriteClient.java:187)
   
   test step
   
       val df = spark.range(0, 1000).toDF("keyid")
         .withColumn("col3", expr("keyid"))
   
       df.write.format("hudi").
         option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
         option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "col3").
         option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "keyid").
         option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "").
         option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator").
         option(DataSourceWriteOptions.OPERATION_OPT_KEY, "insert").
         option("hoodie.insert.shuffle.parallelism", "4").
         option("hoodie.metadata.enable", "true").
         option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
         .mode(SaveMode.Overwrite).save(basePath)
   
       // upsert same record again
       df.write.format("hudi").
         option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
         option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "col3").
         option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "keyid").
         option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "").
         option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator").
         option(DataSourceWriteOptions.OPERATION_OPT_KEY, "upsert").
         option("hoodie.insert.shuffle.parallelism", "4").
         option("hoodie.metadata.enable", "true").
         option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
         .mode(SaveMode.Append).save(basePath)
   
   org.apache.hudi.exception.HoodieMetadataException: Error syncing to metadata table.org.apache.hudi.exception.HoodieMetadataException: Error syncing to metadata table.
   at org.apache.hudi.client.SparkRDDWriteClient.syncTableMetadata(SparkRDDWriteClient.java:447) at org.apache.hudi.client.AbstractHoodieWriteClient.postCommit(AbstractHoodieWriteClient.java:433) at org.apache.hudi.client.AbstractHoodieWriteClient.commitStats(AbstractHoodieWriteClient.java:187) at org.apache.hudi.client.SparkRDDWriteClient.commit(SparkRDDWriteClient.java:121) at org.apache.hudi.HoodieSparkSqlWriter$.commitAndPerformPostOperations(HoodieSparkSqlWriter.scala:564) at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:230) at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:162) at org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) at org.apache.spark.sql.execution.command.Ex
 ecutedCommandExec.doExecute(commands.scala:86) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
   
   
   
   ## Brief change log
   
   
   ## Verify this pull request
   new UT added
   ## 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.

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 commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a 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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/3182?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 [#3182](https://codecov.io/gh/apache/hudi/pull/3182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9582f7d) into [master](https://codecov.io/gh/apache/hudi/commit/0749cc826a9ef6b04e2063074b7abac0e69414a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0749cc8) will **decrease** coverage by `27.95%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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    #3182       +/-   ##
   =============================================
   - Coverage     44.25%   16.30%   -27.96%     
   + Complexity     4595      474     -4121     
   =============================================
     Files           825      280      -545     
     Lines         36577    10876    -25701     
     Branches       3942     1107     -2835     
   =============================================
   - Hits          16187     1773    -14414     
   + Misses        18642     8945     -9697     
   + Partials       1748      158     -1590     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <0.00%> (-16.45%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.38% <ø> (-48.67%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.44% <ø> (+0.03%)` | :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/3182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...hudi/metadata/HoodieBackedTableMetadataWriter.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldGFkYXRhL0hvb2RpZUJhY2tlZFRhYmxlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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: |
   | [.../apache/hudi/keygen/constant/KeyGeneratorType.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2tleWdlbi9jb25zdGFudC9LZXlHZW5lcmF0b3JUeXBlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/3182/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: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/3182/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 [614 more](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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/3182?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 [0749cc8...9582f7d](https://codecov.io/gh/apache/hudi/pull/3182?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] minihippo commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -318,7 +318,12 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
             createInstantTime);
       }).forEach(status -> {
         HoodieWriteStat writeStat = new HoodieWriteStat();
-        writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        // deal Non-partition table
+        if (partition.isEmpty()) {
+          writeStat.setPath(status.getPath().getName());
+        } else {
+          writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        }

Review comment:
       writeStat.setPath((partition.isEmpty() ? "" : partition + Path.SEPARATOR) + status.getPath().getName());




-- 
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] xiarixiaoyao commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -318,7 +318,12 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
             createInstantTime);
       }).forEach(status -> {
         HoodieWriteStat writeStat = new HoodieWriteStat();
-        writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        // deal Non-partition table
+        if (partition.isEmpty()) {
+          writeStat.setPath(status.getPath().getName());
+        } else {
+          writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        }

Review comment:
       good suggest, thanks




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

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

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



[GitHub] [hudi] xiarixiaoyao commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
##########
@@ -677,4 +677,40 @@ class TestMORDataSource extends HoodieClientTestBase {
 
     assertEquals(partitionCounts("2021/03/03"), count7)
   }
+
+  @Test
+  def testNonPartitionTableWithMetaTableEnabled(): Unit = {
+    val df = spark.range(0, 10).toDF("keyid")
+      .withColumn("col3", expr("keyid"))
+      .withColumn("age", expr("keyid + 1000"))
+
+    df.write.format("hudi").
+      option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).

Review comment:
       yes, thanks。




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

To unsubscribe, e-mail: 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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531) 
   
   <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] vinothchandar commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @xiarixiaoyao could you please rebase


-- 
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] garyli1019 commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -318,7 +318,12 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
             createInstantTime);
       }).forEach(status -> {
         HoodieWriteStat writeStat = new HoodieWriteStat();
-        writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        // deal Non-partition table

Review comment:
       nit: deal with

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala
##########
@@ -677,4 +677,40 @@ class TestMORDataSource extends HoodieClientTestBase {
 
     assertEquals(partitionCounts("2021/03/03"), count7)
   }
+
+  @Test
+  def testNonPartitionTableWithMetaTableEnabled(): Unit = {
+    val df = spark.range(0, 10).toDF("keyid")
+      .withColumn("col3", expr("keyid"))
+      .withColumn("age", expr("keyid + 1000"))
+
+    df.write.format("hudi").
+      option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).

Review comment:
       use `.option` to keep the style consistent?




-- 
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] xiarixiaoyao commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @nsivabalan   fixed . pls review again  thanks


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

To unsubscribe, e-mail: 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9582f7dcde72e98c09fc12e0f69f5beef28dbaea Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550) 
   * b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640) 
   
   <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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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






-- 
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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/3182?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 [#3182](https://codecov.io/gh/apache/hudi/pull/3182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d59d984) into [master](https://codecov.io/gh/apache/hudi/commit/0749cc826a9ef6b04e2063074b7abac0e69414a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0749cc8) will **decrease** coverage by `27.96%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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    #3182       +/-   ##
   =============================================
   - Coverage     44.25%   16.28%   -27.97%     
   + Complexity     4595      473     -4122     
   =============================================
     Files           825      280      -545     
     Lines         36577    10878    -25699     
     Branches       3942     1107     -2835     
   =============================================
   - Hits          16187     1772    -14415     
   + Misses        18642     8947     -9695     
   + Partials       1748      159     -1589     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <0.00%> (-16.45%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.38% <ø> (-48.67%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.40% <ø> (ø)` | |
   
   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/3182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...hudi/metadata/HoodieBackedTableMetadataWriter.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldGFkYXRhL0hvb2RpZUJhY2tlZFRhYmxlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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: |
   | [.../apache/hudi/keygen/constant/KeyGeneratorType.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2tleWdlbi9jb25zdGFudC9LZXlHZW5lcmF0b3JUeXBlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/3182/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: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/3182/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 [613 more](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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/3182?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 [0749cc8...d59d984](https://codecov.io/gh/apache/hudi/pull/3182?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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9582f7dcde72e98c09fc12e0f69f5beef28dbaea Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550) 
   
   <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] leesf commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -376,7 +381,9 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
         if (p.getRight().length > filesInDir.size()) {
           // Is a partition. Add all data files to result.

Review comment:
       we should remove the comment here since it maybe a non-partitioned table.




-- 
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] garyli1019 merged pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   


-- 
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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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






-- 
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] xiarixiaoyao commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   @nsivabalan  thanks, i will do it。


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

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

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



[GitHub] [hudi] minihippo commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -318,7 +318,12 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
             createInstantTime);
       }).forEach(status -> {
         HoodieWriteStat writeStat = new HoodieWriteStat();
-        writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        // deal Non-partition table
+        if (partition.isEmpty()) {
+          writeStat.setPath(status.getPath().getName());
+        } else {
+          writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        }

Review comment:
       writeStat.setPath((partition.isEmpty() ? "" : partition + Path.SEPARATOR) + status.getPath().getName());




-- 
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] xiarixiaoyao commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @nsivabalan  sorry for that,  i forget base the code.  i will check the code again


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

To unsubscribe, e-mail: 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7a4a11a1e28af225c0039f5c4f7994761921e2a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671) 
   * 636c067d8f8ca4675be4b3336c8b40b82058897d 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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531) 
   
   <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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=721",
       "triggerID" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 636c067d8f8ca4675be4b3336c8b40b82058897d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=721) 
   
   <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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/3182?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 [#3182](https://codecov.io/gh/apache/hudi/pull/3182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9582f7d) into [master](https://codecov.io/gh/apache/hudi/commit/0749cc826a9ef6b04e2063074b7abac0e69414a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0749cc8) will **decrease** coverage by `15.77%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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    #3182       +/-   ##
   =============================================
   - Coverage     44.25%   28.47%   -15.78%     
   + Complexity     4595     1262     -3333     
   =============================================
     Files           825      376      -449     
     Lines         36577    14327    -22250     
     Branches       3942     1459     -2483     
   =============================================
   - Hits          16187     4080    -12107     
   + Misses        18642     9952     -8690     
   + Partials       1748      295     -1453     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `22.29% <0.00%> (+5.84%)` | :arrow_up: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.38% <ø> (-48.67%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.44% <ø> (+0.03%)` | :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/3182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...hudi/metadata/HoodieBackedTableMetadataWriter.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldGFkYXRhL0hvb2RpZUJhY2tlZFRhYmxlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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: |
   | [.../apache/hudi/keygen/constant/KeyGeneratorType.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2tleWdlbi9jb25zdGFudC9LZXlHZW5lcmF0b3JUeXBlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/3182/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: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/3182/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 [710 more](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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/3182?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 [0749cc8...9582f7d](https://codecov.io/gh/apache/hudi/pull/3182?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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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






-- 
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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640) 
   * c7a4a11a1e28af225c0039f5c4f7994761921e2a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671) 
   
   <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] xiarixiaoyao commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @vinothchandar  thanks for your reviewe.   UT has moved to HoodieSparkSqlWriterTestsuite, and both mor and cow table is coverd


-- 
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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531) 
   * 9582f7dcde72e98c09fc12e0f69f5beef28dbaea Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550) 
   
   <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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   There are some compilation errors. Can you please check that. 
   
   ```
   [WARNING]  Expected all dependencies to require Scala version: 2.11.12
   [WARNING]  org.apache.hudi:hudi-spark_2.11:0.9.0-SNAPSHOT requires scala version: 2.11.12
   [WARNING]  org.apache.hudi:hudi-spark-client:0.9.0-SNAPSHOT requires scala version: 2.11.12
   [WARNING]  org.apache.hudi:hudi-spark-common_2.11:0.9.0-SNAPSHOT requires scala version: 2.11.12
   [WARNING]  org.apache.hudi:hudi-spark2_2.11:0.9.0-SNAPSHOT requires scala version: 2.11.12
   [WARNING]  com.fasterxml.jackson.module:jackson-module-scala_2.11:2.6.7.1 requires scala version: 2.11.8
   [WARNING] Multiple versions of scala libraries detected!
   [INFO] /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/src/test/java:-1: info: compiling
   [INFO] /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/src/test/scala:-1: info: compiling
   [INFO] Compiling 31 source files to /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/target/test-classes at 1625219965712
   [ERROR] /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala:645: error: overloaded method value option with alternatives:
   [ERROR]   (key: String,value: Double)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: Long)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: Boolean)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: String)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row]
   [ERROR]  cannot be applied to (org.apache.hudi.common.config.ConfigProperty[String], String)
   [ERROR]           .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType)
   [ERROR]            ^
   [ERROR] /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala:660: error: overloaded method value option with alternatives:
   [ERROR]   (key: String,value: Double)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: Long)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: Boolean)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row] <and>
   [ERROR]   (key: String,value: String)org.apache.spark.sql.DataFrameWriter[org.apache.spark.sql.Row]
   [ERROR]  cannot be applied to (org.apache.hudi.common.config.ConfigProperty[String], String)
   [ERROR]           .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType)
   [ERROR]            ^
   [WARNING] /home/travis/build/apache/hudi/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala:56: warning: A try without a catch or finally is equivalent to putting its body in a block; no exceptions are handled.
   [WARNING]     try super.test(testName, testTags: _*)(try testFun finally {
   [WARNING]     ^
   [WARNING] one warning found
   [ERROR] two errors found
   [INFO] ------------------------------------------------------------------------
   [INFO] Reactor Summary:
   ```


-- 
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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640) 
   * c7a4a11a1e28af225c0039f5c4f7994761921e2a 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640) 
   
   <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] garyli1019 merged pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   


-- 
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 commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a 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 commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/3182?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 [#3182](https://codecov.io/gh/apache/hudi/pull/3182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d59d984) into [master](https://codecov.io/gh/apache/hudi/commit/0749cc826a9ef6b04e2063074b7abac0e69414a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0749cc8) will **decrease** coverage by `27.96%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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    #3182       +/-   ##
   =============================================
   - Coverage     44.25%   16.28%   -27.97%     
   + Complexity     4595      473     -4122     
   =============================================
     Files           825      280      -545     
     Lines         36577    10878    -25699     
     Branches       3942     1107     -2835     
   =============================================
   - Hits          16187     1772    -14415     
   + Misses        18642     8947     -9695     
   + Partials       1748      159     -1589     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <0.00%> (-16.45%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.38% <ø> (-48.67%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `58.40% <ø> (ø)` | |
   
   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/3182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...hudi/metadata/HoodieBackedTableMetadataWriter.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldGFkYXRhL0hvb2RpZUJhY2tlZFRhYmxlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...main/java/org/apache/hudi/metrics/HoodieGauge.java](https://codecov.io/gh/apache/hudi/pull/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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: |
   | [.../apache/hudi/keygen/constant/KeyGeneratorType.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2tleWdlbi9jb25zdGFudC9LZXlHZW5lcmF0b3JUeXBlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pache/hudi/client/utils/ConcatenatingIterator.java](https://codecov.io/gh/apache/hudi/pull/3182/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: |
   | [.../hudi/execution/bulkinsert/BulkInsertSortMode.java](https://codecov.io/gh/apache/hudi/pull/3182/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 [613 more](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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/3182?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 [0749cc8...d59d984](https://codecov.io/gh/apache/hudi/pull/3182?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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9582f7dcde72e98c09fc12e0f69f5beef28dbaea Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550) 
   * b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265 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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d59d9848553983cedbad7e2a08d4ac33b725a68a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531) 
   * 9582f7dcde72e98c09fc12e0f69f5beef28dbaea 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7a4a11a1e28af225c0039f5c4f7994761921e2a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671) 
   
   <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] xiarixiaoyao commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   @nsivabalan @garyli1019  could you help me to reivew this pr, thanks


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

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

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



[GitHub] [hudi] xiarixiaoyao commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -376,7 +381,9 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
         if (p.getRight().length > filesInDir.size()) {
           // Is a partition. Add all data files to result.

Review comment:
       thanks, i will do it。




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

To unsubscribe, e-mail: 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @xiarixiaoyao could you please rebase


-- 
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] xiarixiaoyao commented on pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   @nsivabalan @garyli1019  could you help me to reivew this pr, thanks


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

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

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



[GitHub] [hudi] xiarixiaoyao commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @vinothchandar  thanks.  already rebased


-- 
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] xiarixiaoyao commented on a change in pull request #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -318,7 +318,12 @@ private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTa
             createInstantTime);
       }).forEach(status -> {
         HoodieWriteStat writeStat = new HoodieWriteStat();
-        writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        // deal Non-partition table
+        if (partition.isEmpty()) {
+          writeStat.setPath(status.getPath().getName());
+        } else {
+          writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName());
+        }

Review comment:
       good suggest, thanks




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

To unsubscribe, e-mail: 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 #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=531",
       "triggerID" : "d59d9848553983cedbad7e2a08d4ac33b725a68a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=550",
       "triggerID" : "9582f7dcde72e98c09fc12e0f69f5beef28dbaea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=640",
       "triggerID" : "b2cc0dd8817120e5d7ea4894afbef1ae4e0ec265",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671",
       "triggerID" : "c7a4a11a1e28af225c0039f5c4f7994761921e2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=721",
       "triggerID" : "636c067d8f8ca4675be4b3336c8b40b82058897d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c7a4a11a1e28af225c0039f5c4f7994761921e2a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=671) 
   * 636c067d8f8ca4675be4b3336c8b40b82058897d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=721) 
   
   <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] xiarixiaoyao commented on pull request #3182: [HUDI-2089] Fix the bug that metatable cannot support non_partition table

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


   @vinothchandar  thanks.  already rebased


-- 
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 #3182: [HUDI-2089]fix the bug that metatable cannot support non_partition table

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/3182?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 [#3182](https://codecov.io/gh/apache/hudi/pull/3182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9582f7d) into [master](https://codecov.io/gh/apache/hudi/commit/0749cc826a9ef6b04e2063074b7abac0e69414a0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0749cc8) will **decrease** coverage by `41.18%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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   #3182       +/-   ##
   ============================================
   - Coverage     44.25%   3.07%   -41.19%     
   + Complexity     4595      82     -4513     
   ============================================
     Files           825     280      -545     
     Lines         36577   10876    -25701     
     Branches       3942    1107     -2835     
   ============================================
   - Hits          16187     334    -15853     
   + Misses        18642   10516     -8126     
   + Partials       1748      26     -1722     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | hudicli | `?` | |
   | hudiclient | `0.00% <0.00%> (-16.45%)` | :arrow_down: |
   | hudicommon | `?` | |
   | hudiflink | `?` | |
   | hudihadoopmr | `?` | |
   | hudisparkdatasource | `?` | |
   | hudisync | `5.38% <ø> (-48.67%)` | :arrow_down: |
   | huditimelineservice | `?` | |
   | hudiutilities | `9.46% <ø> (-48.95%)` | :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/3182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...hudi/metadata/HoodieBackedTableMetadataWriter.java](https://codecov.io/gh/apache/hudi/pull/3182/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-aHVkaS1jbGllbnQvaHVkaS1jbGllbnQtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL21ldGFkYXRhL0hvb2RpZUJhY2tlZFRhYmxlTWV0YWRhdGFXcml0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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/3182/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: |
   | ... and [660 more](https://codecov.io/gh/apache/hudi/pull/3182/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/3182?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/3182?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 [0749cc8...9582f7d](https://codecov.io/gh/apache/hudi/pull/3182?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