[
https://issues.apache.org/jira/browse/BEAM-6923?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16936291#comment-16936291
]
Ankur Goenka commented on BEAM-6923:
------------------------------------
Gcsutil.java use sets the default buffer size for individual file write to be
64MB when the VM memory is more than 1GB.
Artifact staging tend to upload multiple files in parallel and each upload
reserves 64MB causing this issue.
Couple of potential fixes are,
# Limiting concurrent upload of files to a lower number.
# Limit the gcs util buffer size per file.
# Limit concurrent gcs connections so that it applies to all the file uploads.
1 applies only to artifact staging but theoretically this problem can impact a
pipeline which writes to a bunch of files.
2 has a performance penalty when writing to a single file.
I am in favor of 3 as it should not have any performance penalty and applies to
all the gcs related file IO.
cc: [~robertwb]
> OOM errors in jobServer when using GCS artifactDir
> --------------------------------------------------
>
> Key: BEAM-6923
> URL: https://issues.apache.org/jira/browse/BEAM-6923
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-harness
> Reporter: Lukasz Gajowy
> Assignee: Ankur Goenka
> Priority: Major
> Attachments: Instance counts.png, Paths to GC root.png,
> Telemetries.png, beam6923-flink156.m4v, beam6923flink182.m4v, heapdump
> size-sorted.png
>
>
> When starting jobServer with artifactDir pointing to a GCS bucket:
> {code:java}
> ./gradlew :beam-runners-flink_2.11-job-server:runShadow
> -PflinkMasterUrl=localhost:8081 -PartifactsDir=gs://the-bucket{code}
> and running a Java portable pipeline with the following, portability related
> pipeline options:
> {code:java}
> --runner=PortableRunner --jobEndpoint=localhost:8099
> --defaultEnvironmentType=DOCKER
> --defaultEnvironmentConfig=gcr.io/<my-freshly-built-sdk-harness-image>/java:latest'{code}
>
> I'm facing a series of OOM errors, like this:
> {code:java}
> Exception in thread "grpc-default-executor-3" java.lang.OutOfMemoryError:
> Java heap space
> at
> com.google.api.client.googleapis.media.MediaHttpUploader.buildContentChunk(MediaHttpUploader.java:606)
> at
> com.google.api.client.googleapis.media.MediaHttpUploader.resumableUpload(MediaHttpUploader.java:408)
> at
> com.google.api.client.googleapis.media.MediaHttpUploader.upload(MediaHttpUploader.java:336)
> at
> com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:508)
> at
> com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:432)
> at
> com.google.api.client.googleapis.services.AbstractGoogleClientRequest.execute(AbstractGoogleClientRequest.java:549)
> at
> com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel$UploadOperation.call(AbstractGoogleAsyncWriteChannel.java:301)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745){code}
>
> This does not happen when I'm using a local filesystem for the artifact
> staging location.
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)