[ 
https://issues.apache.org/jira/browse/BEAM-11275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17259320#comment-17259320
 ] 

Kyle Weaver edited comment on BEAM-11275 at 1/6/21, 12:50 AM:
--------------------------------------------------------------

Thanks for providing context. Judging from BEAM-5746 it looks like this support 
has been missing for a long time now.

Prior to the regression reported in BEAM-5746, the stager downloaded the file 
from GCS. But it would definitely be better to defer downloading, as you 
suggested. I think we could improve the existing code by deferring the download 
of HTTP(S) files as well.

I think we may be able to accomplish this by changing environments.py [1] to 
use artifact_types.URL [2] instead of artifact_types.FILE when extra_packages 
are remote.

[~gcasassaez] If you intend to implement this yourself, I would be happy to 
help and review. Otherwise I can add it to my backlog, but it may be some time 
before I get around to it.

[1] 
https://github.com/apache/beam/blob/9f97585160fae644d9b4a7f0dca6558c82c29723/sdks/python/apache_beam/transforms/environments.py#L732
[2] 
https://github.com/apache/beam/blob/9f9dcd6e97b29c020f555bcaa992af56108aaa0d/model/pipeline/src/main/proto/beam_runner_api.proto#L1324


was (Author: ibzib):
Thanks for providing context. Judging from BEAM-5746 it looks like this support 
has been missing for a long time now.

Prior to the regression reported in BEAM-5746, the stager downloaded the file 
from GCS. But it would definitely be better to defer downloading, as you 
suggested. I think we could improve the existing code by deferring the download 
of HTTP(S) files as well.

I think we may be able to accomplish this by changing environments.py [1] to 
use artifact_types.URL [2] instead of artifact_types.FILE.

[~gcasassaez] If you intend to implement this yourself, I would be happy to 
help and review. Otherwise I can add it to my backlog, but it may be some time 
before I get around to it.

[1] 
https://github.com/apache/beam/blob/9f97585160fae644d9b4a7f0dca6558c82c29723/sdks/python/apache_beam/transforms/environments.py#L732
[2] 
https://github.com/apache/beam/blob/9f9dcd6e97b29c020f555bcaa992af56108aaa0d/model/pipeline/src/main/proto/beam_runner_api.proto#L1324

> Support GCS files for extra_requirements argument in Python Beam portable 
> runners
> ---------------------------------------------------------------------------------
>
>                 Key: BEAM-11275
>                 URL: https://issues.apache.org/jira/browse/BEAM-11275
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-universal, sdk-py-core
>            Reporter: Gerard Casas Saez
>            Priority: P2
>
> Currently Portable runners only support locally available files for adding 
> dependencies on remote workers. This can be seen in 
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/stager.py#L429
>  as it uses shutil.copyfile when it detects file is remote and it is not http.
> An easy extension would be to extend _is_remote_path in Stager to detect if 
> the path matches any filesystem and if it does the avoid downloading and let 
> it be copied afterwards. 
> Acceptance criteria:
> - `extra_package` can be a GCS path instead of requiring it to be local only.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to