[
https://issues.apache.org/jira/browse/HUDI-3066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17463437#comment-17463437
]
sivabalan narayanan edited comment on HUDI-3066 at 12/21/21, 7:36 PM:
----------------------------------------------------------------------
hey [~h7kanna] : thanks for all the info. Wanted to clarify few things.
a. Please leave the cleaner configs for metadata as is.for eg:
hoodie.metadata.clean.async=false. we should document this, or should not have
exposed to end user. but as of now, only inline cleaning is supported.
b. May I know how is cleaning executed in data table? from what I infer, its
async and not inline. (hoodie.clean.async=true). So, how else is cleaning
getting executed? Is it a separate job or are you using deltastreamer w/ async
cleaning on?
c. I see some of the cleaner commits in data table are missing from metadata
timeline. So, my hunch here is that, your async cleaner job might have missed
to set the "hoodie.metadata.enable" to true and hence those commits are not
seen in metadata timeline. Ideally for every commit you see in data timeline,
you should see a delta commit in metadata timeline.
d. As mentioned in the migration guide, if you are having any async table
services, you are expected to configure lock providers as well. Did you happen
to configure one? I don't see it from above configs.
e. Occationallly I see rollbacks in metadata table. Would you mind posting the
contents of 1 or two of the rollback.requested meta file.
f. From metadata timeline, I don't see any hfiles. which means compaction never
completed successfully. I don't see any error logs that compaction, nor I could
see compaction.requested meta file in metadata timeline as well. this is very
strange. with default metadata config settings, every 10th commit, compaction
should happen. atleast it should be attempted. If you don't mind, can you post
the contents of basepath/.hoodie/metadata/files/
g. I see you have very aggressive clustering configs. for every commit you are
triggering a clustering. May I know is it just part of testing? bcoz, rewriting
data is costly and in generally users try to cluster once every N commits or
so depending on your pipeline. but triggering a clustering for every write is
not a good approach in general.
really appreciate you providing us with all logs and info. We can definitely
get to the bottom of this and get it fixed if need be
was (Author: shivnarayan):
hey [~h7kanna] : thanks for all the info. Wanted to clarify few things.
a. Please leave the cleaner configs for metadata as is.for eg:
hoodie.metadata.clean.async=false. we should document this, or should not have
exposed to end user. but as of now, only inline cleaning is supported.
b. I see your data table, how is cleaning executed? from what I infer, its
async and not inline. (hoodie.clean.async=true). So, how else is cleaning
getting executed? Is it a separate job or are you using deltastreamer w/ async
cleaning on?
c. I see some of the cleaner commits in data table are missing from metadata
timeline. So, my hunch here is that, your async cleaner job might have missed
to set the "hoodie.metadata.enable" to true and hence those commits are not
seen in metadata timeline. Ideally for every commit you see in data timeline,
you should see a delta commit in metadata timeline.
d. As mentioned in the migration guide, if you are having any async table
services, you are expected to configure lock providers as well. Did you happen
to configure one? I don't see it from above configs.
e. Occationallly I see rollbacks in metadata table. Would you mind posting the
contents of 1 or two of the rollback.requested meta file.
f. From metadata timeline, I don't see any hfiles. which means compaction never
completed successfully. I don't see any error logs that compaction, nor I could
see compaction.requested meta file in metadata timeline as well. this is very
strange. with default metadata config settings, every 10th commit, compaction
should happen. atleast it should be attempted. If you don't mind, can you post
the contents of basepath/.hoodie/metadata/files/
g. I see you have very aggressive clustering configs. for every commit you are
triggering a clustering. May I know is it just part of testing? bcoz, rewriting
data is costly and in generally users try to cluster once every N commits or
so depending on your pipeline. but triggering a clustering for every write is
not a good approach in general.
really appreciate you providing us with all logs and info. We can definitely
get to the bottom of this and get it fixed if need be
> Very slow file listing after enabling metadata for existing tables in 0.10.0
> release
> ------------------------------------------------------------------------------------
>
> Key: HUDI-3066
> URL: https://issues.apache.org/jira/browse/HUDI-3066
> Project: Apache Hudi
> Issue Type: Bug
> Affects Versions: 0.10.0
> Environment: EMR 6.4.0
> Hudi version : 0.10.0
> Reporter: Harsha Teja Kanna
> Assignee: Manoj Govindassamy
> Priority: Blocker
> Labels: performance, pull-request-available
> Fix For: 0.11.0
>
> Attachments: Screen Shot 2021-12-18 at 6.16.29 PM.png, Screen Shot
> 2021-12-20 at 10.05.50 PM.png, Screen Shot 2021-12-20 at 10.17.44 PM.png,
> metadata_timeline.txt, metadata_timeline_archived.txt, stderr_part1.txt,
> stderr_part2.txt, timeline.txt, writer_log.txt
>
>
> After 'metadata table' is enabled, File listing takes long time.
> If metadata is enabled on Reader side(as shown below), it is taking even more
> time per file listing task
> {code:java}
> import org.apache.hudi.DataSourceReadOptions
> import org.apache.hudi.common.config.HoodieMetadataConfig
> val hadoopConf = spark.conf
> hadoopConf.set(HoodieMetadataConfig.ENABLE.key(), "true")
> val basePath = "s3a://datalake-hudi"
> val sessions = spark
> .read
> .format("org.apache.hudi")
> .option(DataSourceReadOptions.QUERY_TYPE.key(),
> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
> .option(DataSourceReadOptions.READ_PATHS.key(),
> s"${basePath}/sessions_by_entrydate/entrydate=2021/*/*/*")
> .load()
> sessions.createOrReplaceTempView("sessions") {code}
> Existing tables (COW) have inline clustering on and have many replace commits.
> Logs seem to suggest the delay is in view.AbstractTableFileSystemView
> resetFileGroupsReplaced function or metadata.HoodieBackedTableMetadata
> Also many log messages in AbstractHoodieLogRecordReader
>
> 2021-12-18 23:17:54,056 INFO view.AbstractTableFileSystemView: Took 4118 ms
> to read 136 instants, 9731 replaced file groups
> 2021-12-18 23:37:46,086 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,090 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.76_0-20-515
> at instant 20211217035105329
> 2021-12-18 23:37:46,090 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,094 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.121_0-57-663',
> fileLen=0}
> 2021-12-18 23:37:46,095 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.20_0-35-613',
> fileLen=0}
> 2021-12-18 23:37:46,095 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,096 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.62_0-34-377
> at instant 20211217022049877
> 2021-12-18 23:37:46,096 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,105 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.86_0-20-362',
> fileLen=0}
> 2021-12-18 23:37:46,109 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.121_0-57-663',
> fileLen=0}
> 2021-12-18 23:37:46,109 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,110 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.77_0-35-590',
> fileLen=0}
> 2021-12-18 23:37:46,112 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.20_0-35-613
> at instant 20211216183448389
> 2021-12-18 23:37:46,112 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,118 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.63_0-56-519',
> fileLen=0}
> 2021-12-18 23:37:46,122 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.86_0-20-362',
> fileLen=0}
> 2021-12-18 23:37:46,122 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,123 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.121_0-57-663
> at instant 20211217090337935
> 2021-12-18 23:37:46,123 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,127 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.77_0-35-590',
> fileLen=0}
> 2021-12-18 23:37:46,127 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,141 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.21_0-56-795',
> fileLen=0}
> 2021-12-18 23:37:46,143 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.63_0-56-519',
> fileLen=0}
> 2021-12-18 23:37:46,143 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,144 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.86_0-20-362
> at instant 20211217045206861
> 2021-12-18 23:37:46,144 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,145 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.77_0-35-590
> at instant 20211217035058289
> 2021-12-18 23:37:46,145 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,162 INFO log.AbstractHoodieLogRecordReader: Scanning log
> file
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.21_0-56-795',
> fileLen=0}
> 2021-12-18 23:37:46,162 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.63_0-56-519
> at instant 20211217022306173
> 2021-12-18 23:37:46,162 INFO s3a.S3AInputStream: Switching to Random IO seek
> policy
> 2021-12-18 23:37:46,162 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge 1
> 2021-12-18 23:37:46,170 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.87_0-35-433',
> fileLen=0}
> 2021-12-18 23:37:46,171 INFO log.HoodieLogFormatReader: Moving to the next
> reader for logfile
> HoodieLogFile\{pathStr='s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.78_0-56-705',
> fileLen=0}
> 2021-12-18 23:37:46,178 INFO log.AbstractHoodieLogRecordReader: Reading a
> data block from file
> s3a://datalake-hudi/sessions/.hoodie/metadata/files/.files-0000_20211216144130775001.log.21_0-56-795
> at instant 20211216184106961
> 2021-12-18 23:37:46,178 INFO log.AbstractHoodieLogRecordReader: Number of
> remaining logblocks to merge
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner: Number of log
> files scanned => 437
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner:
> MaxMemoryInBytes allowed for compaction => 1073741824
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner: Number of
> entries in MemoryBasedMap in ExternalSpillableMap => 165
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner: Total size in
> bytes of MemoryBasedMap in ExternalSpillableMap => 259380
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner: Number of
> entries in BitCaskDiskMap in ExternalSpillableMap => 0
> 2021-12-18 23:38:35,380 INFO log.HoodieMergedLogRecordScanner: Size of file
> spilled to disk => 0
> 2021-12-18 23:38:35,380 INFO metadata.HoodieBackedTableMetadata: Opened 437
> metadata log files (dataset instant=20211218233649435, metadata
> instant=20211218233649435) in 22935 ms
> 2021-12-18 23:38:37,193 INFO metadata.HoodieBackedTableMetadata: Opened 437
> metadata log files (dataset instant=20211218233649435, metadata
> instant=20211218233649435) in 22802 ms
> Sample file listing tasks
> !Screen Shot 2021-12-18 at 6.16.29 PM.png!
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)