[ https://issues.apache.org/jira/browse/MAPREDUCE-7435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17730455#comment-17730455 ]
ASF GitHub Bot commented on MAPREDUCE-7435: ------------------------------------------- mehakmeet commented on code in PR #5519: URL: https://github.com/apache/hadoop/pull/5519#discussion_r1222648403 ########## hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/EntryFileIO.java: ########## @@ -63,7 +63,18 @@ public class EntryFileIO { private static final Logger LOG = LoggerFactory.getLogger( EntryFileIO.class); - public static final int WRITER_SHUTDOWN_TIMEOUT = 60; + /** + * How long should the writer shutdown take? + */ + public static final int WRITER_SHUTDOWN_TIMEOUT_SECONDS = 60; + + /** + * How long should trying to queue a write block before giving up + * with an error? + * This is a safety feature to ensure that if something has gone wrong + * in the queue code the job fails with an error rather than just hangs + */ + public static final int WRITER_QUEUE_PUT_TIMEOUT_MINUTES = 10; Review Comment: Sorry, I think I missed these constants being added, don't you think these should be configurable, just for some kind of fallback sakes, so that these values never cause any issues and are easily changeable? I guess if it waits for this long then, we can assume it's just hanging as well. Your call on it being configurable or not. > ManifestCommitter OOM on azure job > ---------------------------------- > > Key: MAPREDUCE-7435 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-7435 > Project: Hadoop Map/Reduce > Issue Type: Bug > Components: client > Affects Versions: 3.3.5 > Reporter: Steve Loughran > Assignee: Steve Loughran > Priority: Major > Labels: pull-request-available > > I've got some reports of spark jobs OOM if the manifest committer is used > through abfs. > either the manifests are using too much memory, or something is not working > with azure stream memory use (or both). > before proposing a solution, first step should be to write a test to load > many, many manifests, each with lots of dirs and files to see what breaks. > note: we did have OOM issues with the s3a committer, on teragen but those > structures have to include every etag of every block, so the manifest size is > O(blocks); the new committer is O(files + dirs). > {code} > java.lang.OutOfMemoryError: Java heap space > at > org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.readOneBlock(AbfsInputStream.java:314) > at > org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.read(AbfsInputStream.java:267) > at java.io.DataInputStream.read(DataInputStream.java:149) > at > com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539) > at > com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133) > at > com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256) > at com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1656) > at com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1085) > at > com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3585) > at > org.apache.hadoop.util.JsonSerialization.fromJsonStream(JsonSerialization.java:164) > at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:279) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest.load(TaskManifest.java:361) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem.loadTaskManifest(ManifestStoreOperationsThroughFileSystem.java:133) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.lambda$loadManifest$6(AbstractJobOrTaskStage.java:493) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage$$Lambda$231/1813048085.apply(Unknown > Source) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding$$Lambda$217/489150849.apply(Unknown > Source) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.loadManifest(AbstractJobOrTaskStage.java:492) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage.fetchTaskManifest(LoadManifestsStage.java:170) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage.processOneManifest(LoadManifestsStage.java:138) > at > org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage$$Lambda$229/137752948.run(Unknown > Source) > at > org.apache.hadoop.util.functional.TaskPool$Builder.lambda$runParallel$0(TaskPool.java:410) > at > org.apache.hadoop.util.functional.TaskPool$Builder$$Lambda$230/467893357.run(Unknown > Source) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > 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:750) > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org