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 2020/07/20 13:17:49 UTC

[GitHub] [hudi] ssomuah opened a new issue #1852: [SUPPORT]

ssomuah opened a new issue #1852:
URL: https://github.com/apache/hudi/issues/1852


   **Describe the problem you faced**
   
   Write performance degrades over time 
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1.Create an unpartitoned MOR table
   2.Use it for a few days
   
   **Expected behavior**
   
   Write performance should not degrade over time 
   **Environment Description**
   
   * Hudi version :  Master @ 3b9a305 https://github.com/apache/hudi/tree/3b9a30528bd6a6369181702303f3384162b04a7f
   
   * Spark version : 2.4.4
   
   * Hive version : N/A
   
   * Hadoop version : 2.7.3
   
   * Storage (HDFS/S3/GCS..) : ABFSS
   
   * Running on Docker? (yes/no) : no 
   
   
   **Additional context**
   
   The MOR table has a single partition. 
   It's a spark streaming application with 5 minute batches. 
   Intially it runs and completes batches in the duration. But over time the time for batches to complete increases. 
   From the spark ui we can see that most of the time is being taken actually writing the files. 
   
   <img width="1584" alt="Screen Shot 2020-07-17 at 2 03 35 PM" src="https://user-images.githubusercontent.com/2061955/87941642-7023e980-ca69-11ea-9f7a-9d801a9be131.png">
   
   
   And looking at the thread dump of the executors they are almost always spending their time listing files. 
   
   I think the reason for this is we have an extremely high number of files in the single partition folder. 
   
   An ls on the folder is showing about 45,000 files. 
   
   The other odd thing is that when we look at the write tasks in the spark ui. There 
   are several tasks that seem to have tiny numbers of records in them. 
   
   <img width="1507" alt="Screen Shot 2020-07-20 at 9 15 35 AM" src="https://user-images.githubusercontent.com/2061955/87941812-b11bfe00-ca69-11ea-9e77-32cf83f6e2e1.png">
   
   
   
   We can see compaction taking place so it's not clear why we still have so many files. 
   <img width="1369" alt="Screen Shot 2020-07-17 at 2 04 33 PM" src="https://user-images.githubusercontent.com/2061955/87941819-b4af8500-ca69-11ea-8f0a-9c23df08052e.png">
   
   
   The table config is 
   
         .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
         .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, HoodieTableType.MERGE_ON_READ.name)
         .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, s"$META_COLUMN.version")
         .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, s"$META_COLUMN.key")
         .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, s"$META_COLUMN.partition")
         .option(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY,
           "com.myCustompayloadClass")
         .option(HoodieCompactionConfig.PAYLOAD_CLASS_PROP,
           "com.myCustompayloadClass")
         .option(HoodieWriteConfig.UPSERT_PARALLELISM, 32)
         .option(HoodieWriteConfig.INSERT_PARALLELISM, 32)
         .option(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, 3)
         .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, 12 )
         .option(HoodieCompactionConfig.INLINE_COMPACT_PROP, true)
         .option(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES, String.valueOf(256 * 1024 * 1024))
         .option(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES, String.valueOf(256 * 1024 * 1024))
         .option(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC, "snappy")
   
   We're using our own payload class that decides what to keep based on a timestamp in the message and not latest. 
   
   **Stacktrace**
   
   
   StackTrace of list operation where we are spending a lot of time. 
   
   sun.net.www.protocol.https.HttpsURLConnectionImpl.getResponseCode(HttpsURLConnectionImpl.java:352)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation.processResponse(AbfsHttpOperation.java:259)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(AbfsRestOperation.java:167)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.execute(AbfsRestOperation.java:124)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.services.AbfsClient.listPath(AbfsClient.java:180)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.listFiles(AzureBlobFileSystemStore.java:549)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.listStatus(AzureBlobFileSystemStore.java:628)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.listStatus(AzureBlobFileSystemStore.java:532)
   shaded.databricks.v20180920_b33d810.org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.listStatus(AzureBlobFileSystem.java:344)
   org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1517)
   org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1557)
   org.apache.hudi.common.fs.HoodieWrapperFileSystem.listStatus(HoodieWrapperFileSystem.java:487)
   org.apache.hudi.common.fs.FSUtils.getAllLogFiles(FSUtils.java:409)
   org.apache.hudi.common.fs.FSUtils.getLatestLogVersion(FSUtils.java:420)
   org.apache.hudi.common.fs.FSUtils.computeNextLogVersion(FSUtils.java:434)
   org.apache.hudi.common.model.HoodieLogFile.rollOver(HoodieLogFile.java:115)
   org.apache.hudi.common.table.log.HoodieLogFormatWriter.<init>(HoodieLogFormatWriter.java:101)
   org.apache.hudi.common.table.log.HoodieLogFormat$WriterBuilder.build(HoodieLogFormat.java:249)
   org.apache.hudi.io.HoodieAppendHandle.createLogWriter(HoodieAppendHandle.java:291)
   org.apache.hudi.io.HoodieAppendHandle.init(HoodieAppendHandle.java:141)
   org.apache.hudi.io.HoodieAppendHandle.doAppend(HoodieAppendHandle.java:197)
   org.apache.hudi.table.action.deltacommit.DeltaCommitActionExecutor.handleUpdate(DeltaCommitActionExecutor.java:77)
   org.apache.hudi.table.action.commit.BaseCommitActionExecutor.handleUpsertPartition(BaseCommitActionExecutor.java:246)
   org.apache.hudi.table.action.commit.BaseCommitActionExecutor.lambda$execute$caffe4c4$1(BaseCommitActionExecutor.java:102)
   org.apache.hudi.table.action.commit.BaseCommitActionExecutor$$Lambda$192/1449069739.call(Unknown Source)
   org.apache.spark.api.java.JavaRDDLike$$anonfun$mapPartitionsWithIndex$1.apply(JavaRDDLike.scala:105)
   
   
   


----------------------------------------------------------------
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] bvaradar closed issue #1852: [SUPPORT]

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


   


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   1. I'm trying this now. 
   2. The stack trace is the one I provided above. 


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   I don't see any exceptions in the driver logs or executor logs. 
   
   I see these two warnings in driver logs
   ```
   20/07/21 13:12:28 WARN IncrementalTimelineSyncFileSystemView: Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing
   ```
   ```
   20/07/21 13:12:29 WARN CleanPlanner: Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed since last cleaned at 20200721032203. New Instant to retain : Option{val=[20200721032203__commit__COMPLETED]}
   ```
   
   These are the contests of the timeline 
   [dot_hoodie_folder.txt](https://github.com/apache/hudi/files/4954820/dot_hoodie_folder.txt)
   
   The timeline only has files from the current day but I see log files in the data folder from over a week ago. 
   
   
   Do you have any ideas of what might be causing so many log files


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   Only 1 compaction.inflight now
   


----------------------------------------------------------------
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] ssomuah edited a comment on issue #1852: [SUPPORT]

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


   @bvaradar I think the issue I'm facing is due to configuration, but I can't pinpoint what it is. 
   
   I'm ending up with an extremely large number of files fo a single partition merge on read table. 
   
   I have tens of thousands of log files which I would have thought would get compacted into parquet at some point. 
   
   what volume of updates are working well for merge on read tables today?


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   What do you mean by "runs serially with ingestion"? My understanding was that inline compaction happened in the same flow as writing so an inline compaction would simply slow down ingestion.
   
    ===> Yes, that is what I meant. Inline Compaction would run after ingestion but not in parallel. You can use #1752 to have it run concurrently.
   
   Does INLINE_COMPACT_NUM_DELTA_COMMITS_PROP refer to the number of commits retained in general, or the number of commits for a record?
   
   ==> INLINE_COMPACT_NUM_DELTA_COMMITS_PROP refers to number of ingestion (deltacommits) between 2 compaction runs. 
   
   I see in the timeline I have several clean.requested and clean.inflight, how can I get these to actually complete?
   
   ==> If it is in inflight state alone, there could be errors when Hudi is trying to cleanup. Please look for exceptions in driver logs. Cleaner run should be run automatically by default. Also, any pending clean operations will automatically get picked up in next ingestion. So, it must have been failing for some reasons. You can turn on logs to see what is happening.
   
   Is it possible to force a compaction of the existing log files.
   
   ===> Yes, by configuring INLINE_COMPACT_NUM_DELTA_COMMITS_PROP. You can set it to 1 to have aggressive compaction. 
   
   


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   ```
   And looking at the thread dump of the executors they are almost always spending their time listing files.
   ```
   
   This looks surprising to me. file listing for finding latest file versions for index lookup and writing happens in driver (concurrently within embedded-service). If the executors have trouble connecting to the driver, then executor would list them. Do you see any non-fatal exceptions when writing ? Can you also paste the timeline (listing of .hoodie folder) 
   


----------------------------------------------------------------
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] ssomuah edited a comment on issue #1852: [SUPPORT]

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


   I don't see any exceptions in the driver logs or executor logs. 
   
   I see these two warnings in driver logs
   ```
   20/07/21 13:12:28 WARN IncrementalTimelineSyncFileSystemView: Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing
   ```
   ```
   20/07/21 13:12:29 WARN CleanPlanner: Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed since last cleaned at 20200721032203. New Instant to retain : Option{val=[20200721032203__commit__COMPLETED]}
   ```
   
   These are the contests of the timeline 
   [dot_hoodie_folder.txt](https://github.com/apache/hudi/files/4954820/dot_hoodie_folder.txt)
   
   The timeline only has files from the current day but I see log files in the data folder from over a week ago, do you have any idea what might be causing so many log files


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   Sorry, I did not realize that. Let me check and get back


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   What do you mean by "runs serially with ingestion"? My understanding was that inline compaction happened in the same flow as writing so an inline compaction would simply slow down ingestion. 
   
   Does INLINE_COMPACT_NUM_DELTA_COMMITS_PROP refer to the number of commits retained in general, or the number of commits for a record? 
   
   I see in the timeline I have several clean.requested and clean.inflight, how can I get these to actually complete?
   
   What determines how many log files are created in each batch for a MOR 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] bvaradar commented on issue #1852: [SUPPORT]

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


   We have a jira : https://issues.apache.org/jira/browse/HUDI-1015 to improve/avoid  listing. I have added this case to the jira.


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   @ssomuah : Looking at the commit metadata, it is the case where your updates are spread across a large number of files. For example, in latest commit, 334 files sees updates whereas only one file is newly created due to inserts. It looks like this is the nature of your workload. 
   
   If your record key has some sort of ordering, then you can initially bootstrap using "bulk-insert" which would sort and write the data in record-key order. This can potentially help reduce the number of files getting updated if each batch of writes have similar ordering.  You can also try recreating the dataset with larger parquet file size and higher small file limit and async compactions (more frequent to keep the number of active log files in check). 
   
   However, you are basically trying to reduce the number of files getting appended at the expense of more data getting appended to a single file. This is a general upsert problem due to the nature of your workload.


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   Ended up creating a new jira : https://issues.apache.org/jira/browse/HUDI-1119 as this has different cause.


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   MacBook-Pro:hudi balaji.varadarajan$ grep -c '\.clean.requested'  ~/Downloads/dot_hoodie_folder.txt 
   16
   MacBook-Pro:hudi balaji.varadarajan$ grep -c '\.deltacommit.requested'  ~/Downloads/dot_hoodie_folder.txt 
   266
   MacBook-Pro:hudi balaji.varadarajan$ grep -c '\.compaction.requested'  ~/Downloads/dot_hoodie_folder.txt 
   44
   MacBook-Pro:hudi balaji.varadarajan$ grep -c '\.compaction.inflight'  ~/Downloads/dot_hoodie_folder.txt 
   44
   MacBook-Pro:hudi balaji.varadarajan$ grep -c '\.commit'  ~/Downloads/dot_hoodie_folder.txt 
   19
   
   1. I can see that there are many compactions that are in inflight status but have not completed.  Can you add this patch : https://github.com/apache/hudi/pull/1857 to retry failed compactions automatically ?
   2. Those 2 warnings are fine and one of them implies that the embedded timeline-server is on. Can you provide the stack trace where you deduced that most time is spent on listing ?
   


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   Hi Balaji, I think I've narrowed down my issue somewhat for my MOR table. 
   
   I started again with a fresh table and the initial commits make sense, but after a time I notice It's consistently trying to write 300+ files. 
   
   <img width="964" alt="Screen Shot 2020-07-24 at 1 15 17 PM" src="https://user-images.githubusercontent.com/2061955/88417393-da14f980-cdaf-11ea-87ab-63f3aafade83.png">
   
   <img width="1398" alt="Screen Shot 2020-07-24 at 1 15 36 PM" src="https://user-images.githubusercontent.com/2061955/88417402-de411700-cdaf-11ea-85dd-c10c405851d3.png">
   
   <img width="1411" alt="Screen Shot 2020-07-24 at 1 15 52 PM" src="https://user-images.githubusercontent.com/2061955/88417424-e5682500-cdaf-11ea-9c4b-534e27d80c45.png">
   
   
   The individual tasks don't take that long so I think if I could reduce the number of files it's trying to write it would help. 
   <img width="1409" alt="Screen Shot 2020-07-24 at 1 16 03 PM" src="https://user-images.githubusercontent.com/2061955/88417487-fca71280-cdaf-11ea-9fc0-10a8a074501c.png">
   
   
   I can also see from the cli that whether it's doing a compaction or a delta commit I still seem to be writing the same number of files for a fraction of the data. 
   <img width="1307" alt="Screen Shot 2020-07-24 at 1 21 36 PM" src="https://user-images.githubusercontent.com/2061955/88417841-aa1a2600-cdb0-11ea-808f-d66595af91ea.png">
   
   
   Is there something I can tune to reduce the number of files it breaks the data into?
   
   hoodie.logfile.max.size is 256MB
   hoodie.parquet.max.file.size is 256MB
   hoodie.parquet.compression.ratio is the default .35


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   I updated to master @ 743ef322b88d90be9775b889f6381925cdda5f35 and then applied the patch you linked above. 
   
   The first batch that ran had several "RunCompactionActionExecutor" 
   
    I'm still consistently seeing long batches 
   
   <img width="1592" alt="Screen Shot 2020-07-23 at 8 45 49 AM" src="https://user-images.githubusercontent.com/2061955/88288372-b07da480-ccc1-11ea-9dc6-4e0a0675244a.png">
   
   
   <img width="1151" alt="Screen Shot 2020-07-23 at 8 46 07 AM" src="https://user-images.githubusercontent.com/2061955/88288414-b96e7600-ccc1-11ea-92f8-ba1671466164.png">
   
   
   <img width="1581" alt="Screen Shot 2020-07-23 at 8 46 17 AM" src="https://user-images.githubusercontent.com/2061955/88288429-c0958400-ccc1-11ea-9617-81bffd01a8af.png">
   
   
   <img width="1583" alt="Screen Shot 2020-07-23 at 8 46 33 AM" src="https://user-images.githubusercontent.com/2061955/88288446-c5f2ce80-ccc1-11ea-9032-0c251d2f94d9.png">
   
   
   The contents of the  timeline folder are now. 
   
   [dot_hoodie_folder_v2.txt](https://github.com/apache/hudi/files/4966070/dot_hoodie_folder_v2.txt)
   
   
   I think the root of my issue is that I have tons of log files which don't seem to get compacted. 
   
   


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   @ssomuah : Regarding the patch, it is meant to ensure all pending compactions are completed.  Regarding the slowness, we are working on general and S3 specific performance improvements on the write side  which should be part of next release : 0.6.0


----------------------------------------------------------------
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] ssomuah edited a comment on issue #1852: [SUPPORT]

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


   @bvaradar I think the issue I'm facing is due to configuration, but I can't pinpoint what it is. 
   
   I'm ending up with an extremely large number of files fo a single partition merge on read table. 
   
   I have tens of thousands of log files which I would have thought would get compacted into parquet at some point. 


----------------------------------------------------------------
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] ssomuah commented on issue #1852: [SUPPORT]

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


   @bvaradar I think the issue I'm facing is due to configuration, but I can't pinpoint what it is. 
   
   I'm ending up with an extremely large number of files fo a single partition merge on read table. 
   
   I have tens of thousand of log files which I would have thought would get compacted into parquet at some point. 


----------------------------------------------------------------
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] ssomuah edited a comment on issue #1852: [SUPPORT]

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


   @bvaradar I think the issue I'm facing is due to configuration, but I can't pinpoint what it is. 
   
   I'm ending up with an extremely large number of files fo a single partition merge on read table. 
   
   I have tens of thousands of log files which I would have thought would get compacted into parquet at some point. 
   
   what volume of updates is working well for merge on read tables today?


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   @ssomuah : 
   Such a large number of log files indicates your compaction frequency (INLINE_COMPACT_NUM_DELTA_COMMITS_PROP)  is conservative. Many of these log files could also be belonging to older file versions which will be cleaned by Cleaner ( https://cwiki.apache.org/confluence/display/HUDI/FAQ#FAQ-WhatdoestheHudicleanerdo). 
   
   In addition, note that inline compaction which runs serially with ingestion. We have a working PR which lets compaction run concurrently with ingestion : https://github.com/apache/hudi/pull/1752
   Now that pending compactions have finished, you can setup concurrent compaction with the above PR ( config : hoodie.datasource.compaction.async.enable=true). 
   
   
   
   


----------------------------------------------------------------
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] ssomuah edited a comment on issue #1852: [SUPPORT]

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


   What do you mean by "runs serially with ingestion"? My understanding was that inline compaction happened in the same flow as writing so an inline compaction would simply slow down ingestion. 
   
   Does INLINE_COMPACT_NUM_DELTA_COMMITS_PROP refer to the number of commits retained in general, or the number of commits for a record? 
   
   I see in the timeline I have several clean.requested and clean.inflight, how can I get these to actually complete?
   
   What determines how many log files are created in each batch for a MOR table?
   
   EDIT:
   Is it possible to force a compaction of the existing log files.


----------------------------------------------------------------
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] bvaradar commented on issue #1852: [SUPPORT]

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


   Closing this ticket as it was answered.


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