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 2021/05/10 13:59:54 UTC

[GitHub] [hudi] t0il3ts0ap opened a new issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

t0il3ts0ap opened a new issue #2934:
URL: https://github.com/apache/hudi/issues/2934


   **Describe the problem you faced**
   
   My aim is to read an existing hudi table (COW) using deltastreamer, do some transformations and write it to another ( fresh ) table. I am using deltastreamer so as check-pointing can be automated.
   
   Relevant hudi configs used for deltastreamer 
   ```
   --hoodie-conf hoodie.parquet.compression.codec=snappy 
   --table-type COPY_ON_WRITE 
   --source-class org.apache.hudi.utilities.sources.HoodieIncrSource 
   --hoodie-conf hoodie.deltastreamer.source.hoodieincr.path=s3://poc-bucket/raw-data/customer_service/credit_analysis_data 
   --hoodie-conf hoodie.deltastreamer.source.hoodieincr.partition.extractor.class=org.apache.hudi.hive.NonPartitionedExtractor 
   --hoodie-conf hoodie.deltastreamer.source.hoodieincr.partition.fields='' 
   --hoodie-conf hoodie.deltastreamer.source.hoodieincr.num_instants=1 
   --enable-sync 
   --checkpoint 0
   ```
   
   
   The first run of deltastreamer failed with 
   ```
   Exception in thread "main" org.apache.spark.sql.AnalysisException: Path does not exist: s3://poc-bucket/raw-data/customer_service/credit_analysis_data/default/c67b4ac1-4597-4896-81c5-dc70b2f62892-1_0-23-13659_20210508062123.parquet;
   	at org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:764)
   	at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
   	at scala.collection.immutable.List.foreach(List.scala:392)
   	at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
   	at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
   	at scala.collection.immutable.List.flatMap(List.scala:355)
   	at org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:751)
   	at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:580)
   	at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:405)
   	at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:297)
   	at org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:286)
   	at scala.Option.getOrElse(Option.scala:189)
   	at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:286)
   	at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:755)
   	at org.apache.hudi.IncrementalRelation.buildScan(IncrementalRelation.scala:151)
   	at org.apache.spark.sql.execution.datasources.DataSourceStrategy.apply(DataSourceStrategy.scala:313)
   	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$1(QueryPlanner.scala:63)
   	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
   	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
   	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
   	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
   	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:69)
   	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
   	at scala.collection.TraversableOnce.$anonfun$foldLeft$1(TraversableOnce.scala:162)
   	at scala.collection.TraversableOnce.$anonfun$foldLeft$1$adapted(TraversableOnce.scala:162)
   	at scala.collection.Iterator.foreach(Iterator.scala:941)
   	at scala.collection.Iterator.foreach$(Iterator.scala:941)
   ``` 
   
   The original table is couple of months old. At any moment I find commits spanning over last 3 days in its .hoodie directory. 
   Surprisingly, the parquet file mentioned in most of commits does not exist.
   
   I am able to obtain same error when trying to run hudi incremental query in spark-shell.
   
   * Hudi version : 0.7.0
   
   * Spark version : 3.0.2 with scala 2.12
   
   * Storage (HDFS/S3/GCS..) :S3
   
   * 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.

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



[GitHub] [hudi] jsbali commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap Do raise the PR for the same if it is not too much work and we can let the Hudi folks decide what makes sense for Hudi. My changes are mostly in IncrementalRelation. Will also try to wrap my PR within a day or two. 


-- 
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] t0il3ts0ap edited a comment on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

Posted by GitBox <gi...@apache.org>.
t0il3ts0ap edited a comment on issue #2934:
URL: https://github.com/apache/hudi/issues/2934#issuecomment-928866105


   I have also worked on the same changes already. @vingov, @jsbali Let me know, if you have not yet started then I can raise a pr for 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.

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

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



[GitHub] [hudi] t0il3ts0ap commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @n3nash Sorry for late reply, was away on vacation
   1. We run one instance of deltastreamer job every 2 hrs on source table. Each run at the max sources 6 million records using --source-limit parameter.
   2. ~ 12 runs of deltastreamer. Not sure if results in 12 commits.  ( Please share some knowledge or documentation here. I apologize that I dont know this. ) 
   3. Please assume defaults for this. I have not modified cleaner policy for deltastreamer. 
   4. This is failing in first run itself. I checked manually, there are no files for oldest commits. 
   
   A sample. case. 
   Lets say I open .hoodie directory for a COW table with no partition
   <img width="1172" alt="Screenshot 2021-05-28 at 3 29 26 PM" src="https://user-images.githubusercontent.com/8509512/119967062-9be8f780-bfc9-11eb-9bca-87cb8fc4fd07.png">
   Now, I open this commit file `20210524175014.commit` to find following data:
   ```
   {
     "partitionToWriteStats" : {
       "default" : [ {
         "fileId" : "890de7c3-7f0d-4586-8e66-83a9f8d9c106-0",
         "path" : "default/890de7c3-7f0d-4586-8e66-83a9f8d9c106-0_0-23-13521_20210524175014.parquet",
         "prevCommit" : "20210524160842",
         "numWrites" : 325368,
         "numDeletes" : 0,
         "numUpdateWrites" : 0,
         "numInserts" : 115,
         "totalWriteBytes" : 32817083,
         "totalWriteErrors" : 0,
         "tempPath" : null,
         "partitionPath" : "default",
         "totalLogRecords" : 0,
         "totalLogFilesCompacted" : 0,
         "totalLogSizeCompacted" : 0,
         "totalUpdatedRecordsCompacted" : 0,
         "totalLogBlocks" : 0,
         "totalCorruptLogBlock" : 0,
         "totalRollbackBlocks" : 0,
         "fileSizeInBytes" : 32817083
       } ]
     },
     "compacted" : false,
     "extraMetadata" : {
       "schema" : "{\"type\":\"record\",\"name\":\"hoodie_source\",\"namespace\":\"hoodie.source\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"created_at\",\"type\":[\"string\",\"null\"]},{\"name\":\"created_by\",\"type\":[\"string\",\"null\"]},{\"name\":\"last_modified_by\",\"type\":[\"string\",\"null\"]},{\"name\":\"version\",\"type\":[\"long\",\"null\"]},{\"name\":\"address_reference_id\",\"type\":[\"string\",\"null\"]},{\"name\":\"city\",\"type\":[\"string\",\"null\"]},{\"name\":\"current\",\"type\":[\"boolean\",\"null\"]},{\"name\":\"line_one\",\"type\":[\"string\",\"null\"]},{\"name\":\"line_two\",\"type\":[\"string\",\"null\"]},{\"name\":\"permanent\",\"type\":[\"boolean\",\"null\"]},{\"name\":\"pin_code\",\"type\":[\"string\",\"null\"]},{\"name\":\"source\",\"type\":[\"string\",\"null\"]},{\"name\":\"state\",\"type\":[\"string\",\"null\"]},{\"name\":\"type\",\"type\":[\"string\",\"null\"]},{\"name\":\"email_address\",\"type\":[\"string\",\"null\"]},{\"name\":\
 "collection_case_id\",\"type\":[\"long\",\"null\"]},{\"name\":\"updated_date\",\"type\":[\"string\",\"null\"]},{\"name\":\"updated_at\",\"type\":[\"string\",\"null\"]},{\"name\":\"latitude\",\"type\":[\"double\",\"null\"]},{\"name\":\"longitude\",\"type\":[\"double\",\"null\"]},{\"name\":\"__lsn\",\"type\":[\"long\",\"null\"]},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"boolean\",\"null\"]}]}",
       "deltastreamer.checkpoint.key" : "collection_service.public.addresses,0:1066373,1:1064984,2:1065562,3:1070617,4:1067374,5:1066110"
     },
     "operationType" : "UPSERT",
     "fileIdAndRelativePaths" : {
       "890de7c3-7f0d-4586-8e66-83a9f8d9c106-0" : "default/890de7c3-7f0d-4586-8e66-83a9f8d9c106-0_0-23-13521_20210524175014.parquet"
     },
     "totalRecordsDeleted" : 0,
     "totalLogRecordsCompacted" : 0,
     "totalLogFilesCompacted" : 0,
     "totalCompactedRecordsUpdated" : 0,
     "totalLogFilesSize" : 0,
     "totalScanTime" : 0,
     "totalCreateTime" : 0,
     "totalUpsertTime" : 5089
   }
   ```
   
   Now , this `default/890de7c3-7f0d-4586-8e66-83a9f8d9c106-0_0-23-13521_20210524175014.parquet` mentioned in the commit is missing. This error is raised only while doing [incremental query ](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java )
   
   But if I just search the prefix `890de7c3-7f0d-4586-8e66-83a9f8d9c106` there are many parquet files which I can see
   
   
   Right now, I have overridden `HoodieIncrSource.java` when running deltastreamer for target table to make it work.
   Added some logic like below, 
   If no checkpoint is found (basically this is first run), do a snapshot query and save last commit as checkpoint. 
   If checkpoint is found carry on with incremental query as usual.
   
   What this benefits us with is that we are always doing incremental query on newer commits rather than older commits. Newer commits's parquet file is present. Also it solves the problem of taking snapshot automatically, when reading from a hudi table for the first time.
   


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



[GitHub] [hudi] nsivabalan commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @jsbali / @t0il3ts0ap : Did either of you folks got a chance to upstream your changes. 


-- 
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] t0il3ts0ap commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   I have also worked on the same changes already. @jsbali Let me know, if you have not yet started then I can raise a pr for 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.

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

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



[GitHub] [hudi] jsbali commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @nsivabalan Just pushed my changes https://github.com/apache/hudi/pull/3946


-- 
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] n3nash commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap Can you please share some more information so I can help you. 
   
   1. What is the rate of new data written to the source table ? Are there many updates to the source table ?
   2. How many commits happen on the source table every day ?
   3. What are your settings for`hoodie.cleaner.policy` and corresponding value for that policy (for eg. hoodie.cleaner.commits.retained) ?
   4. How often are you running the deltastreamer to generate the downstream fresh table ?


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



[GitHub] [hudi] nsivabalan closed issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #2934:
URL: https://github.com/apache/hudi/issues/2934


   


-- 
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] vinothchandar commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   cc @vingov as well. 


-- 
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] t0il3ts0ap edited a comment on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

Posted by GitBox <gi...@apache.org>.
t0il3ts0ap edited a comment on issue #2934:
URL: https://github.com/apache/hudi/issues/2934#issuecomment-928866105


   I have also worked on the same changes already. @vingov, @jsbali Let me know, if you have not yet started then I can raise a pr for 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.

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

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



[GitHub] [hudi] n3nash commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap You can read this blog -> https://github.com/apache/hudi/pull/2967 (soon to be merged) that describes how you can play with source table number of commits and cleaner configs to ensure that if your source table is moving faster than the downstream table, you don't end up cleaning the versioned files in the source by the time the downstream table wants to read it. Let me know if you need further information. 


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



[GitHub] [hudi] nsivabalan commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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






-- 
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] vinothchandar commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   I will keep this open, to track and tag against JIRAs that exist to have more graceful fallback to either listing or metadata table to handle this scenario.


-- 
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] vingov commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @jsbali is working on a fix when we hit this edge case, the delta streamer will do a snapshot query and update the `deltastreamer.checkpoint.key` to the latest commit so that the next run will be incremental based on the latest commit time.
   
   @jsbali - Can you please upstream your changes to OSS? 


-- 
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] vingov commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap - sure, I will confirm in 2 days after checking with @jsbali.
   
   He already has the changes in our internal codebase, he just needs to upstream 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.

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

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



[GitHub] [hudi] vingov removed a comment on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

Posted by GitBox <gi...@apache.org>.
vingov removed a comment on issue #2934:
URL: https://github.com/apache/hudi/issues/2934#issuecomment-932829205


   @t0il3ts0ap - sure, I will confirm in 2 days after checking with @jsbali.
   
   He already has the changes in our internal codebase, he just needs to upstream 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.

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

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



[GitHub] [hudi] n3nash commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap Can you please share some more information so I can help you. 
   
   1. What is the rate of new data written to the source table ? Are there many updates to the source table ?
   2. How many commits happen on the source table every day ?
   3. What are your settings for`hoodie.cleaner.policy` and corresponding value for that policy (for eg. hoodie.cleaner.commits.retained) ?
   4. How often are you running the deltastreamer to generate the downstream fresh table ?


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



[GitHub] [hudi] t0il3ts0ap commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   I have also worked on the same changes already. @jsbali Let me know, if you have not yet started then I can raise a pr for 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.

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

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



[GitHub] [hudi] jsbali commented on issue #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   @t0il3ts0ap Do raise the PR for the same if it is not too much work and we can let the Hudi folks decide what makes sense for Hudi. My changes are mostly in IncrementalRelation. Will also try to wrap my PR within a day or two. 


-- 
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 #2934: [SUPPORT] Parquet file does not exist when trying to read hudi table incrementally

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


   Will close out the ticket as this is expected with interplays between archival and incremental queries. and since we have a patch addressing 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.

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

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