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 2019/12/30 08:26:24 UTC

[GitHub] [incubator-hudi] hddong opened a new pull request #1157: Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

hddong opened a new pull request #1157: Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157
 
 
   ## What is the purpose of the pull request
   
   *Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata*
   
   ## Brief change log
   
     - *Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata*
   
   ## Verify this pull request
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574945396
 
 
   @hddong : If possible try to see if you can run it again ? pull master,  rebase and force-push 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-572830782
 
 
   @bvaradar All has been fixed, please review again. And `operationType` now be `String`, due to AVRO-1676.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362728805
 
 

 ##########
 File path: hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
 ##########
 @@ -510,21 +515,21 @@ private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, Workload
   /**
    * Commit changes performed at the given commitTime marker.
    */
-  public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
-    return commit(commitTime, writeStatuses, Option.empty());
+  public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses, WriteOperationType operationType) {
 
 Review comment:
   As only one hudi write operation is outstanding at a time, can you cache the last operation type in instance variables within HoodieWriteClient object so that users don't need to explicitly pass them in this commit() call

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362727335
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java
 ##########
 @@ -106,6 +108,14 @@ public void setCompacted(Boolean compacted) {
     return filePaths;
   }
 
+  public void setOperateType(WriteOperationType type) {
 
 Review comment:
   rename to operationType along with getters/setters.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365069807
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java
 ##########
 @@ -397,10 +402,81 @@ public void testArchiveCommitCompactionNoHole() throws IOException {
         timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
   }
 
+  @Test
+  public void testArchiveCommitAndDeepCopy() throws IOException {
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
+        .build();
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
+    HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
+    HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf());
+    HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
+        new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
+    HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
+    HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
+    HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf());
+    HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
+        new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf());
+    HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
+    HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf());
+    HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf());
+
+    assertTrue(archiveLog.archiveIfRequired(jsc));
+
+    // read the file
+    Reader reader =
+        HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-1")),
+            HoodieArchivedMetaEntry.getClassSchema());
+    List<IndexedRecord> readRecords = new ArrayList<>();
+    // read the avro blocks and validate the number of records written in each avro block
+    while (reader.hasNext()) {
+      HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
+      List<IndexedRecord> records = blk.getRecords();
+      readRecords.addAll(records);
+    }
+
+    HoodieArchivedMetaEntry metaEntry = (HoodieArchivedMetaEntry)SpecificData.get()
+        .deepCopy(HoodieArchivedMetaEntry.getClassSchema(), readRecords.get(0));
+
+    assertEquals("operationType must be UNKNOWN by default.",
+        metaEntry.getHoodieCommitMetadata().getOperationType(), "UNKNOWN");
+    reader.close();
+  }
+
   private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
     HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
         .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights();
     assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants,
         timeline.countInstants());
   }
+
+  @Test
+  public void testCommitMetadataConverter() {
+    HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
+    hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
+
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-commitMetadata-converter")
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
+        .build();
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
+
+    Class<?> clazz  = HoodieCommitArchiveLog.class;
+    try {
+      Method commitMetadataConverter = clazz.getDeclaredMethod("commitMetadataConverter", HoodieCommitMetadata.class);
+      commitMetadataConverter.setAccessible(true);
+      org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata =
+          (org.apache.hudi.avro.model.HoodieCommitMetadata) commitMetadataConverter.invoke(archiveLog, hoodieCommitMetadata);
+      assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
+    } catch (NoSuchMethodException e) {
+      e.printStackTrace();
 
 Review comment:
   Instead of e.printStackTrace() in all these catch blocks, can you throw these exception for the test to fail

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar merged pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar merged pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362727612
 
 

 ##########
 File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
 ##########
 @@ -82,10 +82,10 @@ object DataSourceWriteOptions {
     * Default: upsert()
     */
   val OPERATION_OPT_KEY = "hoodie.datasource.write.operation"
-  val BULK_INSERT_OPERATION_OPT_VAL = "bulk_insert"
-  val INSERT_OPERATION_OPT_VAL = "insert"
-  val UPSERT_OPERATION_OPT_VAL = "upsert"
-  val DELETE_OPERATION_OPT_VAL = "delete"
+  val BULK_INSERT_OPERATION_OPT_VAL = WriteOperationType.BULK_INSERT.toString
 
 Review comment:
   Let us not change these configuration values as it would cause backwards compatibility issues.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-571921938
 
 
   > @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.
   
   @bvaradar I found it cause by AVRO-1676 and fixed in AVRO-1.8.0. So can i roll back `operateType ` to String type?  It may cause other compatibility problem If upgrade avro.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574427217
 
 
   @bvaradar @vinothchandar Thanks. rebased this and there are no conflicts now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362130717
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java
 ##########
 @@ -46,4 +47,25 @@ public void testPerfStatPresenceInHoodieMetadata() throws Exception {
     Assert.assertTrue(metadata.getTotalScanTime() == 0);
     Assert.assertTrue(metadata.getTotalLogFilesCompacted() > 0);
   }
+
+  @Test
+  public void testCompatibilityWithoutOperateType() throws Exception {
 
 Review comment:
   Thanks for adding the compatibility tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-571521331
 
 
   @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-580653102
 
 
   @bvaradar @vinothchandar please review this 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-593433072
 
 
   @bvaradar As you said keep operationType string here, and i had resolve all conflict.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-573482504
 
 
   > Have some concerns on backwards compatibility.. We don't write commit metadata as avro , so may be this works for now?
   
   @vinothchandar It works for archiving now, no for write commit metadata as avro.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r386701101
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java
 ##########
 @@ -413,4 +417,32 @@ private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expect
     assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants,
         timeline.countInstants());
   }
+
+  @Test
+  public void testCommitMetadataConverter() {
+    HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
+    hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
+
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-commitMetadata-converter")
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
+        .build();
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
+
+    Class<?> clazz  = HoodieCommitArchiveLog.class;
+    try {
+      Method commitMetadataConverter = clazz.getDeclaredMethod("commitMetadataConverter", HoodieCommitMetadata.class);
 
 Review comment:
   @hddong  One final comment : Can you make commitMetadataConverter() in HoodieCommitArchiveLog with default access (instead of private). This way, you dont need to deal with reflection.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365069288
 
 

 ##########
 File path: hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
 ##########
 @@ -98,7 +99,7 @@
   private final transient HoodieMetrics metrics;
   private final transient HoodieCleanClient<T> cleanClient;
   private transient Timer.Context compactionTimer;
-
+  private transient WriteOperationType operationType;
 
 Review comment:
   Can you move this AbstractHoodieWriteClient and use setters/getters in HoodieWriteClient

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365621431
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
 
 Review comment:
   can we add this at the end? is nt that need for this to be backwards compatible?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-569881431
 
 
   > @hddong can you explain why add operation type to HoodieCommitMetadata? thanks
   
   @hmatu It has no special function except for information tagging and tracing at this patch now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-571921938
 
 
   > @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.
   
   @bvaradar I found it cause by [AVRO-1676](https://issues.apache.org/jira/browse/AVRO-1676) and fixed in AVRO-1.8.0. So can i roll back `operateType ` to String type?  It may cause other compatibility problem If upgrade avro.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362728271
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
+         "name":"operateType",
+         "type":["null","string"],
 
 Review comment:
   Can we use enum instead of string type ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-570159839
 
 
   @bvaradar please re-review this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-575021883
 
 
   @bvaradar I tried what you suggest, but there are some other problems. I tried another way and tried rebase locally, it seems ok.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574938954
 
 
   @bvaradar this was weird. In my side, this web page shows there are no conflicts, it's all passed and ready to merge.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574945171
 
 
   <img width="1350" alt="Screen Shot 2020-01-15 at 5 47 57 PM" src="https://user-images.githubusercontent.com/3021376/72486802-ccf3ce00-37c0-11ea-80b1-f91f3f8d038e.png">
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362728404
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
+         "name":"operateType",
+         "type":["null","string"],
+         "default": null
 
 Review comment:
   Also, Can you confirm if the operation type is stored in the avro objects when archiving ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-580693541
 
 
   > > @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.
   > 
   > @bvaradar I found it cause by [AVRO-1676](https://issues.apache.org/jira/browse/AVRO-1676) and fixed in AVRO-1.8.0. So can i roll back `operationType ` to String type? It may cause other compatibility problem If upgrade avro.
   
   Already using `avro-1.8.2` dependency, String type is redundant now.
   
   https://github.com/apache/incubator-hudi/blob/f27c7a16c6d437efaa83e50a7117b83e5201ac49/pom.xml#L96

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574440583
 
 
   @hddong : Did you forgot to push the diff. Still seeing conflicts. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574309467
 
 
   @hddong : lgtm. Once you rebase this PR and resolve conflicts, we can merge

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r386937077
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java
 ##########
 @@ -413,4 +417,32 @@ private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expect
     assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants,
         timeline.countInstants());
   }
+
+  @Test
+  public void testCommitMetadataConverter() {
+    HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
+    hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
+
+    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-commitMetadata-converter")
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
+        .build();
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
+
+    Class<?> clazz  = HoodieCommitArchiveLog.class;
+    try {
+      Method commitMetadataConverter = clazz.getDeclaredMethod("commitMetadataConverter", HoodieCommitMetadata.class);
 
 Review comment:
   > @hddong One final comment : Can you make commitMetadataConverter() in HoodieCommitArchiveLog with default access (instead of private). This way, you dont need to deal with reflection.
   
   @bvaradar Change to publice access, because `TestHoodieCommitArchiveLog` and `HoodieCommitArchiveLog` not in the same package path.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365110224
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java
 ##########
 @@ -397,10 +402,81 @@ public void testArchiveCommitCompactionNoHole() throws IOException {
         timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
   }
 
+  @Test
+  public void testArchiveCommitAndDeepCopy() throws IOException {
 
 Review comment:
   > Is this method used to test the enum issue in Avro? If so, you can remove it.
   
   Yes, be removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r366499639
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
 
 Review comment:
   Agree with Vinoth, Avro schema evolution expects new fields to be appended. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365628090
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
 
 Review comment:
   > is nt that need for this to be backwards compatible
   
   Move it to the end. IMO, it has no impact on compatibility.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362130184
 
 

 ##########
 File path: hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
 ##########
 @@ -171,7 +172,7 @@ public static SparkConf registerClasses(SparkConf conf) {
       JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, jsc, table);
       metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
       indexTimer = null;
-      return upsertRecordsInternal(taggedRecords, commitTime, table, true);
+      return upsertRecordsInternal(taggedRecords, commitTime, table, true, Type.UPSERT);
 
 Review comment:
   There is an enum OperationType in HoodieWriteClient. It is more fine-grained in that it is able to distinguish between PREPPED and non-PREDDED version of operations. Can we use that enum instead of Type. You can move it to a separate enum class in the package org.apache.hudi.common.model and name it as WriteOperationType
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-572835965
 
 
   IMO, `WriteOperationType` duplicate to `OperationType`, keep one is a better way.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hmatu edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hmatu edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-580693541
 
 
   > > @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.
   > 
   > @bvaradar I found it cause by [AVRO-1676](https://issues.apache.org/jira/browse/AVRO-1676) and fixed in AVRO-1.8.0. So can i roll back `operationType ` to String type? It may cause other compatibility problem If upgrade avro.
   
   Already using `avro-1.8.2` dependency, String type is redundant now, Enum type is enough.
   
   https://github.com/apache/incubator-hudi/blob/f27c7a16c6d437efaa83e50a7117b83e5201ac49/pom.xml#L96

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-591598480
 
 
   @hmatu : Sorry for the long delay. Was offline for a while. I agree that operationType should be string in both avro and json structure to keep it simple. Once you make the final change and resolve merge conflict, I will merge this PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-569882681
 
 
   @bvaradar thanks for your review and suggestion, I will modify later.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] codecov-io commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-593852722
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=h1) Report
   > Merging [#1157](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-hudi/commit/2d040145810b8b14c59c5882f9115698351039d1?src=pr&el=desc) will **decrease** coverage by `0.09%`.
   > The diff coverage is `80%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-hudi/pull/1157/graphs/tree.svg?width=650&token=VTTXabwbs2&height=150&src=pr)](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##             master    #1157     +/-   ##
   ===========================================
   - Coverage     67.09%   66.99%   -0.1%     
     Complexity      223      223             
   ===========================================
     Files           333      334      +1     
     Lines         16216    16269     +53     
     Branches       1659     1660      +1     
   ===========================================
   + Hits          10880    10900     +20     
   - Misses         4598     4632     +34     
   + Partials        738      737      -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/table/HoodieCommitArchiveLog.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdGFibGUvSG9vZGllQ29tbWl0QXJjaGl2ZUxvZy5qYXZh) | `75% <ø> (ø)` | `0 <0> (ø)` | :arrow_down: |
   | [.../apache/hudi/client/AbstractHoodieWriteClient.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpZW50L0Fic3RyYWN0SG9vZGllV3JpdGVDbGllbnQuamF2YQ==) | `70.89% <100%> (-4.24%)` | `0 <0> (ø)` | |
   | [...apache/hudi/common/model/HoodieCommitMetadata.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZUNvbW1pdE1ldGFkYXRhLmphdmE=) | `54.58% <100%> (-3.5%)` | `0 <0> (ø)` | |
   | [...java/org/apache/hudi/client/HoodieWriteClient.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpZW50L0hvb2RpZVdyaXRlQ2xpZW50LmphdmE=) | `69.77% <100%> (+0.54%)` | `0 <0> (ø)` | :arrow_down: |
   | [...g/apache/hudi/common/model/WriteOperationType.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL1dyaXRlT3BlcmF0aW9uVHlwZS5qYXZh) | `57.14% <57.14%> (ø)` | `0 <0> (?)` | |
   | [...n/java/org/apache/hudi/common/model/HoodieKey.java](https://codecov.io/gh/apache/incubator-hudi/pull/1157/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZUtleS5qYXZh) | `94.44% <0%> (+5.55%)` | `0% <0%> (ø)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=footer). Last update [2d04014...e34024f](https://codecov.io/gh/apache/incubator-hudi/pull/1157?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-571921938
 
 
   > @bvaradar Operation type is stored in the avro objects when archiving, but there are a error here, it throw `ClassCastException: org.apache.avro.generic.GenericData$EnumSymbol cannot be cast to org.apache.hudi.avro.model.WriteOperationType` with `deepCopy` block in 'show archived commit stats' . Can you give any suggestion when you're free.
   
   @bvaradar I found it cause by [AVRO-1676](https://issues.apache.org/jira/browse/AVRO-1676) and fixed in AVRO-1.8.0. So can i roll back `operationType ` to String type?  It may cause other compatibility problem If upgrade avro.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-572918303
 
 
   @bvaradar Thanks very much for your review, all of them be addressed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-593433072
 
 
   @bvaradar As you said keep operationType string here, and I had resolve all conflict.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365069713
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCommitArchiveLog.java
 ##########
 @@ -397,10 +402,81 @@ public void testArchiveCommitCompactionNoHole() throws IOException {
         timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
   }
 
+  @Test
+  public void testArchiveCommitAndDeepCopy() throws IOException {
 
 Review comment:
   Is this method used to test the enum issue in Avro? If so, you can remove 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362130449
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java
 ##########
 @@ -106,6 +150,14 @@ public void setCompacted(Boolean compacted) {
     return filePaths;
   }
 
+  public void setOperateType(Type type) {
 
 Review comment:
   We also need to add the enum type to the avro schema hudi-common/src/main/avro/HoodieCommitMetadata.avsc
   
   For archiving, we use the avro class org.apache.hudi.avro.model.HoodieCommitMetadata instead of org.apache.hudi.common.model.HoodieCommitMetadata
   
   HoodieCommitArchiveLog.commitMetadataConverter needs to change to copy the operation types to avro objects.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r362727261
 
 

 ##########
 File path: hudi-common/src/main/avro/HoodieCommitMetadata.avsc
 ##########
 @@ -129,6 +129,11 @@
          }],
          "default": null
       },
+      {
+         "name":"operateType",
 
 Review comment:
   nit: operationtionType

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hmatu commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-569622110
 
 
   @hddong can you explain why add operation type to HoodieCommitMetadata? 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-574027065
 
 
   @hddong thanks! avro works based on field positions, so reordering them was my concern. Thanks for addressing this. over to @bvaradar 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#discussion_r365069589
 
 

 ##########
 File path: hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
 ##########
 @@ -492,6 +501,11 @@ protected void postCommit(HoodieCommitMetadata metadata, String instantTime,
     }
   }
 
+  @Override
+  protected void updateOperationType(HoodieCommitMetadata metadata) {
 
 Review comment:
   If we move the operationType instance to base-class, you no longer need this overridden method. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata

Posted by GitBox <gi...@apache.org>.
hddong edited a comment on issue #1157: [HUDI-332]Add operation type (insert/upsert/bulkinsert/delete) to HoodieCommitMetadata
URL: https://github.com/apache/incubator-hudi/pull/1157#issuecomment-572830782
 
 
   @bvaradar All has been fixed, please review again. And `operationType` now be `String`, due to [AVRO-1676](https://issues.apache.org/jira/browse/AVRO-1676) .

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services