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 2022/04/21 06:26:02 UTC

[GitHub] [hudi] lihuahui5683 opened a new issue, #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

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

   **Describe the problem you faced**
   The following exception occurs when hive incremental query hudi xxx_rt : 
   ```
   22/04/21 10:43:58 INFO scheduler.TaskSetManager: Lost task 0.0 in stage 1.0 (TID 8) on Hadoop02, executor 7: java.lang.ClassCastException (org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim) [duplicate 3]
   22/04/21 10:43:58 INFO cluster.YarnClusterScheduler: Removed TaskSet 1.0, whose tasks have all completed, from pool 
   22/04/21 10:43:58 ERROR client.RemoteDriver: Failed to run client job 60806c1e-f2b0-4ee5-bbab-46f8238f3493
   java.util.concurrent.ExecutionException: Exception thrown by job
   	at org.apache.spark.JavaFutureActionWrapper.getImpl(FutureAction.scala:337)
   	at org.apache.spark.JavaFutureActionWrapper.get(FutureAction.scala:342)
   	at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:404)
   	at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:365)
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   	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:748)
   Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
   Aborting TaskSet 1.0 because task 2 (partition 2)
   cannot run anywhere due to node and executor blacklist.
   Most recent failure:
   Lost task 2.0 in stage 1.0 (TID 10, Hadoop02, executor 7): java.lang.ClassCastException: org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim
   	at org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:205)
   	at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat$HoodieCombineFileInputFormatShim.getRecordReader(HoodieCombineHiveInputFormat.java:979)
   	at org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.getRecordReader(HoodieCombineHiveInputFormat.java:556)
   	at org.apache.spark.rdd.HadoopRDD$$anon$1.liftedTree1$1(HadoopRDD.scala:272)
   	at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:271)
   	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:225)
   	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:96)
   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
   	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
   	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   	at org.apache.spark.scheduler.Task.run(Task.scala:121)
   	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:407)
   	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1408)
   	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:413)
   	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:748)
   ```
   I also set the following parameters: 
   ```
   add jar hdfs://mycluster/hudi/jars/hudi-hadoop-mr-bundle-0.10.0.jar;
   set hive.input.format = org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;
   set hoodie.role_sync_hive.consume.mode=INCREMENTAL;
   set hoodie.role_sync_hive.consume.max.commits=3;
   set mapreduce.input.fileinputformat.split.maxsize=128;
   set hive.fetch.task.conversion=none;
   set hoodie.role_sync_hive.consume.start.timestamp=20220420143200507;
   ```
   The query statement is as follows:
   ```
   select * from role_sync_hive_rt where `_hoodie_commit_time` > '20220420143200507';
   ```
   
   **Environment Description**
   
   * Hudi version : 0.10.0
   
   * Spark version : 2.4.0_cdh6.3.2
   
   * Hive version : 2.1.1_cdh6.3.2
   
   * Hadoop version : 3.0.0_cdh6.3.2
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) : no
   
   


-- 
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 #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118203393

   Can you show the create table descriptor in Hive CLI to see the table input format ? What input format does the table use ?


-- 
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] codope commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
codope commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1109718599

   I believe this could be related to HUDI-1036. 
   @lihuahui5683 possible to upgrade to Hive 2.3.x and use Hadoop 2.7.x?


-- 
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 #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118340000

   Did you notice that there is chinese document here ? https://www.yuque.com/docs/share/879349ce-7de4-4284-9126-9c2a3c93a91d?#mmgZE
   


-- 
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] lihuahui5683 commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
lihuahui5683 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118384126

   I have read this document, but I still can't solve this problem. Is it possible that the Hive version is too low?


-- 
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] yihua commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1113638433

   @danny0405 Do you have any idea on resolving the Hive issue in CDH?


-- 
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 #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1119273063

   Can you ask for help in the dingTalk group, some people might solve this problem already.


-- 
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] lihuahui5683 commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
lihuahui5683 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118455860

   yes,I set `hive.input.format=org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;`
   The picture below is the process I performed:
   ![image](https://user-images.githubusercontent.com/31878723/166916026-0906b907-7892-4f1d-b3c7-a588630f5e7c.png)
   ![image](https://user-images.githubusercontent.com/31878723/166916213-c3c3ec10-2f20-48b4-8596-e609d2a3f7ad.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.

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

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


[GitHub] [hudi] lihuahui5683 closed issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
lihuahui5683 closed issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim
URL: https://github.com/apache/hudi/issues/5382


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

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

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


[GitHub] [hudi] nsivabalan commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1126596458

   hey @lihuahui5683 @danny0405 : a. if there was any workaround, can you folks add an FAQ. b. if some code changes are required, can we consider upstreaming it. c. if the issue is resolved, please feel free to close out the github issue.


-- 
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] lihuahui5683 commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
lihuahui5683 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1112097288

   @codope Thanks for your reply.
   I used CDH6.3.2. I have tried to upgrade Hive 2.3.4 and hive 3.1.2. The same exception occurs in Hive 2.3.4 and Hive 2.1.1. Hive 3.1.2 causes metaStore startup failure.Do you have a way to upgrade the CDH Hive version?
   The _ro table can be used normally.
   
   I used flink CDC(flink-1.13.6) to read mysql to write hudi, and used hive to query hudi _rt table.
   The steps are as follows:
   ```
   ./bin/sql-client.sh
   set sql-client.execution.result-mode=tableau;
   set execution.checkpointing.interval=30sec;
   
   create table role (
     channel_id bigint,
     org_game_id string,
     pf smallint,
     org_server_id string,
     org_user_id string,
     org_role_id string,
     role_name string,
     role_lvl int,
     vip_lvl int,
     role_strength bigint,
     role_create_time timestamp(3),
     first_pay_lvl int,
     first_pay_date bigint,
     pay_money decimal(10,2),
     pay_num bigint,
     last_pay_date bigint,
     last_login_date bigint,
     PRIMARY KEY(channel_id,org_game_id,pf,org_user_id,org_server_id,org_role_id) NOT ENFORCED
   ) WITH (
     'connector'='mysql-cdc',
     'hostname'='192.168.20.76',
     'port'='3306',
     'username'='root',
     'password'='k8U@*hy4icomxz',
     'database-name'='test',
     'table-name'='role_info',
     'server-time-zone'='Asia/Shanghai',
     'scan.startup.mode'='initial',
     'scan.snapshot.fetch.size'='1024',
     'debezium.min.row.count.to.stream.result'='500'
   );
   
   create view role_v as select *, date_format(role_create_time, 'yyyy-MM-dd') as dt from role;
   
   create table role_sync_hive (
     channel_id bigint,
     org_game_id string,
     pf smallint,
     org_server_id string,
     org_user_id string,
     org_role_id string,
     role_name string,
     role_lvl int,
     vip_lvl int,
     role_strength bigint,
     role_create_time timestamp(3),
     first_pay_lvl int,
     first_pay_date bigint,
     pay_money decimal(10,2),
     pay_num bigint,
     last_pay_date bigint,
     last_login_date bigint,
     dt string,
     PRIMARY KEY(channel_id,org_game_id,pf,org_user_id,org_server_id,org_role_id) NOT ENFORCED
   )
   partitioned by (dt, org_game_id)
   with (
     'connector'='hudi',
     'path'='hdfs://mycluster/hudi/role_sync_hive',
     'hoodie.datasource.write.recordkey.field'='channel_id.org_game_id.pf.org_user_id.org_server_id.org_role_id',
     'write.precombine.field'='role_create_time',
     'write.tasks'='4',
     'compaction.tasks'='4',
     'write.rate.limit'='2000',
     'table.type'='MERGE_ON_READ',
     'compaction.async.enabled'='true',
     'compaction.schedule.enabled'='true',
     'compaction.trigger.strategy'='num_commits',
     'compaction.delta_commits'='1',
     'compaction.delta_seconds'='60',
     'changelog.enabled'='true',
     'read.streaming.enabled'='true',
     'read.streaming.check-interval'='3',
     'hive_sync.enable'='true',
     'hive_sync.mode'='hms',
     'hive_sync.metastore.uris' = 'thrift://192.168.20.77:9083',
     'hive_sync.table'='role_sync_hive',
     'hive_sync.db'='hudi',
     'hive_sync.username'='hive',
     'hive_sync.password'='',
     'hive_sync.support_timestamp'='true'
   );
   
   insert into role_sync_hive select channel_id, org_game_id, pf, org_server_id, org_user_id, org_role_id, role_name, role_lvl, vip_lvl, role_strength, role_create_time, first_pay_lvl, first_pay_date, pay_money, pay_num, last_pay_date, last_login_date, dt from role_v;
   ```
   Hive Query statement:
   ```
   add jar hdfs://mycluster/hudi/jars/hudi-hadoop-mr-bundle-0.10.0.jar;
   set hive.input.format = org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;
   set hoodie.role_sync_hive.consume.mode=INCREMENTAL;
   set hoodie.role_sync_hive.consume.max.commits=3;
   set mapreduce.input.fileinputformat.split.maxsize=128;
   set hive.fetch.task.conversion=none;
   set hoodie.role_sync_hive.consume.start.timestamp=20220420143200507;
   
   select count(*) from role_sync_hive_rt where `_hoodie_commit_time` > '20220420143200507';
   ```


-- 
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] lihuahui5683 commented on issue #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
lihuahui5683 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118248202

   ```sql
   CREATE EXTERNAL TABLE `role_sync_hive_rt`(
      `_hoodie_commit_time` string COMMENT '',
      `_hoodie_commit_seqno` string COMMENT '',
      `_hoodie_record_key` string COMMENT '',
      `_hoodie_partition_path` string COMMENT '',
      `_hoodie_file_name` string COMMENT '',
      `_hoodie_operation` string COMMENT '',
      `channel_id` bigint COMMENT '',
      `org_game_id` string COMMENT '',
      `pf` int COMMENT '',
      `org_server_id` string COMMENT '',  
      `org_user_id` string COMMENT '',
      `org_role_id` string COMMENT '',
      `role_name` string COMMENT '',
      `role_lvl` int COMMENT '',
      `vip_lvl` int COMMENT '',
      `role_strength` bigint COMMENT '', 
      `role_create_time` bigint COMMENT '',
      `first_pay_lvl` int COMMENT '',
      `first_pay_date` bigint COMMENT '', 
      `pay_money` decimal(10,2) COMMENT '',
      `pay_num` bigint COMMENT '',
      `last_pay_date` bigint COMMENT '', 
      `last_login_date` bigint COMMENT '')
    PARTITIONED BY (
      `dt` string COMMENT '')
    ROW FORMAT SERDE
      'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
    WITH SERDEPROPERTIES (
      'hoodie.query.as.ro.table'='false',
      'path'='hdfs://dalan/hudi/role_sync_hive')
    STORED AS INPUTFORMAT
      'org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat'
    OUTPUTFORMAT
      'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
    LOCATION
      'hdfs://dalan/hudi/role_sync_hive' 
    TBLPROPERTIES (
      'last_commit_time_sync'='20220505150827925',
      'spark.sql.sources.provider'='hudi',
      'spark.sql.sources.schema.numPartCols'='1',
      'spark.sql.sources.schema.numParts'='1',
      'spark.sql.sources.schema.part.0'='{"type":"struct","fields":[{"name":"_hoodie_commit_time","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_commit_seqno","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_record_key","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_partition_path","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_file_name","type":"string","nullable":true,"metadata":{}},{"name":"_hoodie_operation","type":"string","nullable":true,"metadata":{}},{"name":"channel_id","type":"long","nullable":false,"metadata":{}},{"name":"org_game_id","type":"string","nullable":false,"metadata":{}},{"name":"pf","type":"integer","nullable":false,"metadata":{}},{"name":"org_server_id","type":"string","nullable":false,"metadata":{}},{"name":"org_user_id","type":"string","nullable":false,"metadata":{}},{"name":"org_role_id","type":"string","nullable":false,"metadata":{}},{"name":"role_name","type":"string","nullable":true,"metad
 ata":{}},{"name":"role_lvl","type":"integer","nullable":true,"metadata":{}},{"name":"vip_lvl","type":"integer","nullable":true,"metadata":{}},{"name":"role_strength","type":"long","nullable":true,"metadata":{}},{"name":"role_create_time","type":"timestamp","nullable":true,"metadata":{}},{"name":"first_pay_lvl","type":"integer","nullable":true,"metadata":{}},{"name":"first_pay_date","type":"long","nullable":true,"metadata":{}},{"name":"pay_money","type":"decimal(10,2)","nullable":true,"metadata":{}},{"name":"pay_num","type":"long","nullable":true,"metadata":{}},{"name":"last_pay_date","type":"long","nullable":true,"metadata":{}},{"name":"last_login_date","type":"long","nullable":true,"metadata":{}},{"name":"dt","type":"string","nullable":true,"metadata":{}}]}',
      'spark.sql.sources.schema.partCol.0'='dt',
      'transient_lastDdlTime'='1651734483');
   ```


-- 
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 #5382: [SUPPORT] org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit cannot be cast to org.apache.hadoop.hive.shims.HadoopShimsSecure$InputSplitShim

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5382:
URL: https://github.com/apache/hudi/issues/5382#issuecomment-1118422593

   Did you set up the input format manually ? Hive 2.1.1 is not low, many people in out dingTalk group uses that.


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