satishkotha commented on a change in pull request #1929:
URL: https://github.com/apache/hudi/pull/1929#discussion_r486525140



##########
File path: 
hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -94,6 +95,11 @@
   public static final String BULKINSERT_SORT_MODE = 
"hoodie.bulkinsert.sort.mode";
   public static final String DEFAULT_BULKINSERT_SORT_MODE = 
BulkInsertSortMode.GLOBAL_SORT
       .toString();
+  public static final String DELETE_MARKER_FIELD_PROP = 
"hoodie.write.delete.marker.field";

Review comment:
       Is this needed for this change? what is this used for? 

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -117,7 +118,17 @@ public boolean commitStats(String instantTime, 
List<HoodieWriteStat> stats, Opti
     if (extraMetadata.isPresent()) {
       extraMetadata.get().forEach(metadata::addMetadata);
     }
-    metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
+    String schema = config.getSchema();
+    if (config.updatePartialFields()) {
+      try {
+        TableSchemaResolver resolver = new 
TableSchemaResolver(table.getMetaClient());
+        schema = resolver.getTableAvroSchemaWithoutMetadataFields().toString();
+      } catch (Exception e) {
+        // ignore exception.
+        schema = config.getSchema();

Review comment:
       We are potentially reducing schema here,  so I think this can lead to 
issues.  Can we throw error? At the least, can you add a LOG here to make sure 
this gets noticed? 

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -117,7 +118,17 @@ public boolean commitStats(String instantTime, 
List<HoodieWriteStat> stats, Opti
     if (extraMetadata.isPresent()) {
       extraMetadata.get().forEach(metadata::addMetadata);
     }
-    metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
+    String schema = config.getSchema();
+    if (config.updatePartialFields()) {
+      try {
+        TableSchemaResolver resolver = new 
TableSchemaResolver(table.getMetaClient());

Review comment:
       Do you need to create resolver again? Does config.getLastSchema() work 
here?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java
##########
@@ -73,7 +74,11 @@
     } else {
       gReader = null;
       gWriter = null;
-      readSchema = upsertHandle.getWriterSchemaWithMetafields();
+      if (table.getConfig().updatePartialFields() && 
!StringUtils.isNullOrEmpty(table.getConfig().getLastSchema())) {
+        readSchema = new 
Schema.Parser().parse(table.getConfig().getLastSchema());

Review comment:
       similar comment as before. if we make config.getSchema() to always track 
full table schema, this can be simplified.
   

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
##########
@@ -90,9 +92,19 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String 
instantTime, String
    * @param config Write Config
    * @return
    */
-  protected static Pair<Schema, Schema> 
getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config) {
+  protected static Pair<Schema, Schema> 
getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config, 
HoodieTable hoodieTable) {
     Schema originalSchema = new Schema.Parser().parse(config.getSchema());
     Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema);
+    boolean updatePartialFields = config.updatePartialFields();
+    if (updatePartialFields) {
+      try {
+        TableSchemaResolver resolver = new 
TableSchemaResolver(hoodieTable.getMetaClient());

Review comment:
       This is only applicable for MergeHandle if i understand correctly. Do 
you think its better to override this in MergeHandle?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java
##########
@@ -73,7 +74,11 @@
     } else {
       gReader = null;
       gWriter = null;
-      readSchema = upsertHandle.getWriterSchemaWithMetafields();
+      if (table.getConfig().updatePartialFields() && 
!StringUtils.isNullOrEmpty(table.getConfig().getLastSchema())) {
+        readSchema = new 
Schema.Parser().parse(table.getConfig().getLastSchema());
+      } else {
+        readSchema = upsertHandle.getWriterSchemaWithMetafields();

Review comment:
       we are also calling getWriterSchemaWithMetafields in other places in 
this class (example: line 163). Dont we need to read getLastSchema() there?

##########
File path: 
hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
##########
@@ -237,6 +238,9 @@ protected void finalizeWrite(String instantTime, 
List<HoodieWriteStat> stats, Ho
    * By default, return the writer schema in Write Config for storing in 
commit.
    */
   protected String getSchemaToStoreInCommit() {
+    if (config.updatePartialFields() && 
!StringUtils.isNullOrEmpty(config.getLastSchema())) {

Review comment:
       This check is repeated in multiple places. I usually find this pattern 
error-prone. Is it possible to reorganize? For example, we always expect 
config.getSchema() to represent full table schema. We add new 
config.getUpdateSchema() that tracks partial fields that are being updated.  
   
   With that approach, I think we can use getUpdateSchema only in 
MergeHandle/helpers. Storing schema would work as before as we store full table 
schema i.e., config.getSchema()

##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
##########
@@ -192,6 +208,10 @@ protected long getAttemptId() {
 
   protected HoodieFileWriter createNewFileWriter(String instantTime, Path 
path, HoodieTable<T> hoodieTable,
       HoodieWriteConfig config, Schema schema, SparkTaskContextSupplier 
sparkTaskContextSupplier) throws IOException {
-    return HoodieFileWriterFactory.getFileWriter(instantTime, path, 
hoodieTable, config, schema, sparkTaskContextSupplier);
+    if (config.updatePartialFields() && 
!StringUtils.isNullOrEmpty(config.getLastSchema())) {
+      return HoodieFileWriterFactory.getFileWriter(instantTime, path, 
hoodieTable, config, new Schema.Parser().parse(config.getLastSchema()), 
sparkTaskContextSupplier);

Review comment:
       same question as above, is it better to override this only in 
MergeHandle?




----------------------------------------------------------------
This is an automated message from the 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:
us...@infra.apache.org


Reply via email to