rahil-c commented on code in PR #13650:
URL: https://github.com/apache/hudi/pull/13650#discussion_r2289036029


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestSchemaEvolution.java:
##########
@@ -386,7 +386,9 @@ private TableOptions defaultTableOptions(String tablePath) {
         HoodieWriteConfig.EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED.key(), false,
         HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), true,
         HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "true",
-        HoodieMetadataConfig.ENABLE_METADATA_INDEX_PARTITION_STATS.key(), 
"false");
+        HoodieMetadataConfig.ENABLE_METADATA_INDEX_PARTITION_STATS.key(), 
"false",
+        // Disable complex key generator validation so that the writer can 
succeed
+        HoodieWriteConfig.ENABLE_COMPLEX_KEYGEN_VALIDATION.key(), false);

Review Comment:
   Without this change it seems test will fail 
   
   ```
   Caused by: org.apache.flink.util.FlinkException: Global failure triggered by 
OperatorCoordinator for 'stream_write: default_database.t1' (operator 
bfa3bbd691d78791ac299a21f2d951c0).
        at 
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:651)
        at 
org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$1(StreamWriteOperatorCoordinator.java:234)
        at 
org.apache.hudi.sink.utils.NonThrownExecutor.handleException(NonThrownExecutor.java:143)
        at 
org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:134)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750)
   Caused by: org.apache.hudi.exception.HoodieException: Executor executes 
action [request instant time] error
        ... 6 more
   Caused by: org.apache.hudi.exception.HoodieException: This table uses the 
complex key generator with a single record key field. If the table is written 
with Hudi 0.14.1, 0.15.0, 1.0.0, 1.0.1, or 1.0.2 release before, the table may 
potentially contain duplicates due to a breaking change in the key encoding in 
the _hoodie_record_key meta field (HUDI-7001) which is crucial for upserts. 
Please take action based on the mitigation guide before resuming the ingestion 
to the this table. If you're certain that the table is not affected by the key 
encoding change, set `hoodie.write.complex.keygen.validation.enable=false` to 
skip this validation.
        at 
org.apache.hudi.client.BaseHoodieWriteClient.validateComplexKeygen(BaseHoodieWriteClient.java:1705)
        at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommit(BaseHoodieWriteClient.java:1017)
        at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommit(BaseHoodieWriteClient.java:973)
        at 
org.apache.hudi.sink.StreamWriteOperatorCoordinator.startInstant(StreamWriteOperatorCoordinator.java:474)
        at 
org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$handleCoordinationRequest$6(StreamWriteOperatorCoordinator.java:383)
        at 
org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:131)
        ... 3 more
   ```
   
   I think you are right that for new table we likely dont need this validation 
though, since recordkey encoding would only be `fieldName:fieldValue`



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to