You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Leoyzen (via GitHub)" <gi...@apache.org> on 2023/02/02 03:51:40 UTC

[GitHub] [hudi] Leoyzen opened a new issue, #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Leoyzen opened a new issue, #7824:
URL: https://github.com/apache/hudi/issues/7824

   **Describe the problem you faced**
   When enabling metadata on a table which don't has metadata previously, NPE occurs.
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. setup a table with metadata disabled.
   2. restart the job and enabling the metadata option.
   3. NPE occurs
   
   **Expected behavior**
   
   No NPE
   
   **Environment Description**
   
   * Hudi version : 0.13.0-rc1
   
   * Spark version : N/A
   
   * Hive version : 3.1.2
   
   * Hadoop version : 3.1.3
   
   * Storage (HDFS/S3/GCS..) : OSS
   
   * Running on Docker? (yes/no) : yes, HA VVP
   
   
   **Additional context**
   
   
   **Stacktrace**
   
   ```LOG
   023-02-01 00:36:05,435 WARN  org.apache.hudi.metadata.HoodieBackedTableMetadata           [] - Metadata table was not found at path oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/metadata
   2023-02-01 00:36:05,436 INFO  org.apache.hudi.common.table.view.FileSystemViewManager      [] - Creating View Manager with storage type :REMOTE_FIRST
   2023-02-01 00:36:05,436 INFO  org.apache.hudi.common.table.view.FileSystemViewManager      [] - Creating remote first table view
   2023-02-01 00:36:05,438 INFO  org.apache.hudi.client.transaction.lock.LockManager          [] - LockProvider org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider
   2023-02-01 00:36:05,618 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Loading HoodieTableMetaClient from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test
   2023-02-01 00:36:05,635 INFO  org.apache.hudi.common.table.HoodieTableConfig               [] - Loading table properties from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/hoodie.properties
   2023-02-01 00:36:05,647 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Finished Loading Table of type MERGE_ON_READ(version=1, baseFileFormat=PARQUET) from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test
   2023-02-01 00:36:05,776 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201003302911__deltacommit__INFLIGHT]}
   2023-02-01 00:36:05,778 WARN  org.apache.hudi.metadata.HoodieBackedTableMetadataWriter     [] - Cannot initialize metadata table as operation(s) are in progress on the dataset: [[==>20230131214246466__compaction__INFLIGHT], [==>20230131214554966__compaction__REQUESTED], [==>20230131215522662__compaction__REQUESTED], [==>20230131220449842__compaction__REQUESTED], [==>20230201000246991__compaction__REQUESTED], [==>20230201001718618__rollback__INFLIGHT], [==>20230201003302911__deltacommit__INFLIGHT]]
   2023-02-01 00:36:05,778 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Loading HoodieTableMetaClient from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test
   2023-02-01 00:36:05,789 INFO  org.apache.hudi.common.table.HoodieTableConfig               [] - Loading table properties from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/hoodie.properties
   2023-02-01 00:36:05,798 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Finished Loading Table of type MERGE_ON_READ(version=1, baseFileFormat=PARQUET) from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test
   2023-02-01 00:36:05,799 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Loading HoodieTableMetaClient from oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/metadata
   2023-02-01 00:36:05,814 WARN  org.apache.hudi.metadata.HoodieBackedTableMetadata           [] - Metadata table was not found at path oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/metadata
   2023-02-01 00:36:05,938 INFO  org.apache.hadoop.hive.conf.HiveConf                         [] - Found configuration file jar:file:../usrlib/ververica-connector-hudi-1.15-vvr-6.0-hive312-0.13.0-rc1-SNAPSHOT-jar-with-dependencies-20230131234928.jar!/hive-site.xml
   2023-02-01 00:36:06,099 WARN  org.apache.hadoop.hive.conf.HiveConf                         [] - HiveConf of name hive.dummyparam.test.server.specific.config.override does not exist
   2023-02-01 00:36:06,100 WARN  org.apache.hadoop.hive.conf.HiveConf                         [] - HiveConf of name hive.dummyparam.test.server.specific.config.metastoresite does not exist
   
   
   ...........
   
   
   
   
   
   2023-02-01 00:36:24,740 INFO  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Executor executes action [handle write metadata event for instant ] success!
   2023-02-01 00:36:24,800 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Sink: compact_commit (1/1) #0 (b732feaa948ea68f9bf1c0df9689f8f4) switched from INITIALIZING to RUNNING.
   2023-02-01 00:36:24,886 INFO  org.apache.hudi.client.BaseHoodieWriteClient                 [] - Generate a new instant time: 20230201003624886 action: deltacommit
   2023-02-01 00:36:25,295 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201003624920__rollback__REQUESTED]}
   2023-02-01 00:36:25,297 INFO  org.apache.hudi.client.heartbeat.HoodieHeartbeatClient       [] - Received request to start heartbeat for instant time 20230201003624886
   2023-02-01 00:36:25,329 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Creating a new instant [==>20230201003624886__deltacommit__REQUESTED]
   2023-02-01 00:36:25,364 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/20230201003624886.deltacommit.requested
   2023-02-01 00:36:25,394 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_pre/tmp_hudi_hive_test/.hoodie/20230201003624886.deltacommit.inflight
   2023-02-01 00:36:25,435 INFO  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Create instant [20230201003624886] for table [tbl_resp_sink] with type [MERGE_ON_READ]
   2023-02-01 00:36:25,439 INFO  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Executor executes action [initialize instant ] success!
   2023-02-01 00:36:25,746 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - compact_plan_generate (1/1) #0 (06c8d064a1b42253bf7dba8786f881a0) switched from INITIALIZING to FAILED on job-84b59a23-8dfd-41e9-968d-e83d5280df5d-taskmanager-1-1 @ bd-flink011088139163.na63.tbsite.net (dataPort=23011).
   java.lang.NullPointerException: null
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:933) ~[?:?]
   	at org.apache.hudi.table.action.BaseActionExecutor.lambda$writeTableMetadata$2(BaseActionExecutor.java:76) ~[?:?]
   	at org.apache.hudi.common.util.Option.ifPresent(Option.java:97) ~[?:?]
   	at org.apache.hudi.table.action.BaseActionExecutor.writeTableMetadata(BaseActionExecutor.java:76) ~[?:?]
   	at org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.finishRollback(BaseRollbackActionExecutor.java:246) ~[?:?]
   	at org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.runRollback(BaseRollbackActionExecutor.java:114) ~[?:?]
   	at org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.execute(BaseRollbackActionExecutor.java:135) ~[?:?]
   	at org.apache.hudi.table.HoodieFlinkMergeOnReadTable.rollback(HoodieFlinkMergeOnReadTable.java:128) ~[?:?]
   	at org.apache.hudi.table.HoodieTable.rollbackInflightInstant(HoodieTable.java:626) ~[?:?]
   	at org.apache.hudi.table.HoodieTable.rollbackInflightCompaction(HoodieTable.java:600) ~[?:?]
   	at org.apache.hudi.table.HoodieTable.rollbackInflightCompaction(HoodieTable.java:584) ~[?:?]
   	at org.apache.hudi.util.CompactionUtil.lambda$rollbackCompaction$1(CompactionUtil.java:190) ~[?:?]
   	at java.util.ArrayList.forEach(ArrayList.java:1249) ~[?:1.8.0_102]
   	at org.apache.hudi.util.CompactionUtil.rollbackCompaction(CompactionUtil.java:188) ~[?:?]
   	at org.apache.hudi.sink.compact.CompactionPlanOperator.open(CompactionPlanOperator.java:75) ~[?:?]
   	at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:107) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:851) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:798) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:765) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:954) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:923) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:746) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:568) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at java.lang.Thread.run(Thread.java:834) ~[?:1.8.0_102]
   2023-02-01 00:36:25,780 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionFailoverStrategy [] - Calculating tasks to restart to recover the failed task d14c81370450f18851be93e3825a2d06_0.
   2023-02-01 00:36:25,785 INFO  org.apache.flink.runtime.jobmaster.JobMaster                 [] - 14 tasks should be restarted to recover the failed task d14c81370450f18851be93e3825a2d06_0. 
   2023-02-01 00:36:25,785 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Job 1c7f0fe9-c059-491d-86eb-322554d24d0f (84b59a238dfd41e9968de83d5280df5d) switched from state RUNNING to RESTARTING.
   2023-02-01 00:36:25,787 WARN  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Reset the event for task [2]
   2023-02-01 00:36:25,787 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - bucket_write: tbl_resp_sink (3/4) #0 (88f02c5d12f53da0b571729af1aae985) switched from RUNNING to CANCELING.
   2023-02-01 00:36:25,788 WARN  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Reset the event for task [3]
   2023-02-01 00:36:25,788 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - bucket_write: tbl_resp_sink (4/4) #0 (4101dbc91d55a0711db769e51450689c) switched from RUNNING to CANCELING.
   2023-02-01 00:36:25,788 WARN  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Reset the event for task [0]
   2023-02-01 00:36:25,788 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - bucket_write: tbl_resp_sink (1/4) #0 (bdcc0b8232bc1939fd6d29256bab09c3) switched from RUNNING to CANCELING.
   2023-02-01 00:36:25,788 WARN  org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Reset the event for task [1]
   ``
   
   


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

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


[GitHub] [hudi] danny0405 commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1418892653

   Thanks for the feedback, I have created a JIRA issue: https://issues.apache.org/jira/browse/HUDI-5711


-- 
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] danny0405 commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1415240681

   Can you help confirm whether the metadata table path is created?


-- 
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] danny0405 commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1420170722

   @Leoyzen , after an offline huddle with @codope , we make consesus that the restriction is too strict for single-writer senarios, for single writer, the metadata table can be initialized correctly even if there are pending instants on the data set timeline, we would try to improve that like in release 0.13.1 maybe.
   
   For now, you can rollback the pending instants first on the timeline with the HUDI CLI, then restart the job with metadata table enabled, of course, this is a temp solution.


-- 
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] Leoyzen commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "Leoyzen (via GitHub)" <gi...@apache.org>.
Leoyzen commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1415707154

   @danny0405 No, the path doesn't exists.
   After upgrading latest branch(0.13.0-rc1), the exception has changed.
   ```LOG
   2023-02-03 18:54:52,537 [flink-akka.actor.default-dispatcher-39] INFO  org.apache.flink.runtime.jobmaster.JobMaster                 [] - Trying to recover from a global failure.
   org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'bucket_write: tbl_resp_sink' (operator 2b7fd632fd7edc109c5698c0b534dcae).
   	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:597) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$0(StreamWriteOperatorCoordinator.java:196) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.handleException(NonThrownExecutor.java:142) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:133) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1147) ~[?:1.8.0_102]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:622) ~[?:1.8.0_102]
   	at java.lang.Thread.run(Thread.java:834) ~[?:1.8.0_102]
   Caused by: org.apache.hudi.exception.HoodieException: Executor executes action [initialize instant 20230203185127227] error
   	... 6 more
   Caused by: org.apache.hudi.exception.HoodieException: Failed to update metadata
   	at org.apache.hudi.client.HoodieFlinkTableServiceClient.writeTableMetadata(HoodieFlinkTableServiceClient.java:181) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.writeTableMetadata(HoodieFlinkWriteClient.java:279) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:282) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:233) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:111) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:74) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:199) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:548) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:524) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:495) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$initInstant$6(StreamWriteOperatorCoordinator.java:413) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130) ~[?:?]
   	... 3 more
   Caused by: java.lang.IllegalStateException: Metadata table is not fully initialized yet.
   	at org.apache.hudi.common.util.ValidationUtils.checkState(ValidationUtils.java:67) ~[?:?]
   	at org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter.commit(FlinkHoodieBackedTableMetadataWriter.java:109) ~[?:?]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.processAndCommit(HoodieBackedTableMetadataWriter.java:823) ~[?:?]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:890) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkTableServiceClient.writeTableMetadata(HoodieFlinkTableServiceClient.java:179) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.writeTableMetadata(HoodieFlinkWriteClient.java:279) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:282) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:233) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:111) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:74) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:199) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:548) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:524) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:495) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$initInstant$6(StreamWriteOperatorCoordinator.java:413) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130) ~[?:?]
   	... 3 more
   ```
   
   It seems the procedure:
   
   1. The hudi launched and found there isn't any metadata exists, try to create one.
   2. The creation fails because of there is delta commit inflighting.
   3. When the data flushing to the disk, the error occurs(the metadata has not been initilization).
   
   ```LOG
   2023-02-03 18:54:08,449 [pool-24-thread-1] WARN  org.apache.hudi.metadata.HoodieBackedTableMetadataWriter     [] - Cannot initialize metadata table as operation(s) are in progress on the dataset: [[==>20230203184713442__deltacommit__INFLIGHT], [==>20230203185127185__rollback__INFLIGHT], [==>20230203185127227__deltacommit__INFLIGHT]]
   ..........
   2023-02-03 18:54:08,238 [pool-24-thread-1] WARN  org.apache.hudi.metadata.HoodieBackedTableMetadata           [] - Metadata table was not found at path oss://dengine-lake-zjk/cloudcode_pre/dwd_egc_adv_resp_intra/.hoodie/metadata
   ............
   Caused by: java.lang.IllegalStateException: Metadata table is not fully initialized yet.
   ```


-- 
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] Leoyzen commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "Leoyzen (via GitHub)" <gi...@apache.org>.
Leoyzen commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1422076996

   @danny0405 That's a acceptable solution.
   But in our scenario, hudi cli has not been used so far.
   I will wait until next release fixed.


-- 
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] danny0405 commented on issue #7824: [SUPPORT] NPE occurs when enabling metadata on table which does'nt has metadata previously.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #7824:
URL: https://github.com/apache/hudi/issues/7824#issuecomment-1418910757

   It is first introduced in https://github.com/apache/hudi/pull/5222, @codope , can you explain why the metadata table can not be initialized when there are pending instants on the data set timeline?
   
   For table with async table services, like cleaning, compaction and clustering, there is very high possibility that the pending instants exists on the timeline, so how can we then initialize the metadata table correctly?


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