[ 
https://issues.apache.org/jira/browse/BEAM-3883?focusedWorklogId=100273&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-100273
 ]

ASF GitHub Bot logged work on BEAM-3883:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/May/18 21:49
            Start Date: 09/May/18 21:49
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on a change in pull request #5251: 
[BEAM-3883] Refactor and clean dependency.py to make it reusable with artifact 
service
URL: https://github.com/apache/beam/pull/5251#discussion_r187161722
 
 

 ##########
 File path: sdks/python/apache_beam/runners/portability/stager.py
 ##########
 @@ -0,0 +1,556 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+"""Support for installing custom code and required dependencies.
+
+Workflows, with the exception of very simple ones, are organized in multiple
+modules and packages. Typically, these modules and packages have
+dependencies on other standard libraries. Beam relies on the Python
+setuptools package to handle these scenarios. For further details please read:
+https://pythonhosted.org/an_example_pypi_project/setuptools.html
+
+When a runner tries to run a pipeline it will check for a --requirements_file
+and a --setup_file option.
+
+If --setup_file is present then it is assumed that the folder containing the
+file specified by the option has the typical layout required by setuptools and
+it will run 'python setup.py sdist' to produce a source distribution. The
+resulting tarball (a .tar or .tar.gz file) will be staged at the staging
+location specified as job option. When a worker starts it will check for the
+presence of this file and will run 'easy_install tarball' to install the
+package in the worker.
+
+If --requirements_file is present then the file specified by the option will be
+staged in the staging location.  When a worker starts it will check for the
+presence of this file and will run 'pip install -r requirements.txt'. A
+requirements file can be easily generated by running 'pip freeze -r
+requirements.txt'. The reason a runner does not run this automatically is
+because quite often only a small fraction of the dependencies present in a
+requirements.txt file are actually needed for remote execution and therefore a
+one-time manual trimming is desirable.
+
+TODO(silviuc): Staged files should have a job specific prefix.
+To prevent several jobs in the same project stomping on each other due to a
+shared staging location.
+
+TODO(silviuc): Should we allow several setup packages?
+TODO(silviuc): We should allow customizing the exact command for setup build.
+"""
+
+import glob
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tempfile
+
+import pkg_resources
+
+from apache_beam.internal import pickler
+from apache_beam.io.filesystems import FileSystems
+from apache_beam.options.pipeline_options import SetupOptions
+# TODO(angoenka): Remove reference to dataflow internal names
+from apache_beam.runners.dataflow.internal import names
+from apache_beam.utils import processes
+
+# All constants are for internal use only; no backwards-compatibility
+# guarantees.
+
+# Standard file names used for staging files.
+WORKFLOW_TARBALL_FILE = 'workflow.tar.gz'
+REQUIREMENTS_FILE = 'requirements.txt'
+EXTRA_PACKAGES_FILE = 'extra_packages.txt'
+
+# Package names for distributions
+BEAM_PACKAGE_NAME = 'apache-beam'
+
+
+class FileHandler(object):
 
 Review comment:
   The class is needed to group the functionality of copy/download/check_remote 
as these are the 3 things which are used by stager.
   Also file_download needs file_copy as it copies file from remote if its not 
an http url.
   
   FileHandler is not intended to be implemented for every runner as the sdk 
will not directly interact with runner instead will only submit job to a 
job_service and artifacts to an artifact service for both of which we have well 
defined contract. 
   However by proving FileHandler as an argument, we keep the possibility open 
for different adding other means of interacting with files and also support the 
old runners.
   
   I will update the docstring to highlight the usage as suggested.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 100273)
    Time Spent: 4h 20m  (was: 4h 10m)

> Python SDK stages artifacts when talking to job server
> ------------------------------------------------------
>
>                 Key: BEAM-3883
>                 URL: https://issues.apache.org/jira/browse/BEAM-3883
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-py-core
>            Reporter: Ben Sidhom
>            Assignee: Ankur Goenka
>            Priority: Major
>          Time Spent: 4h 20m
>  Remaining Estimate: 0h
>
> The Python SDK does not currently stage its user-defined functions or 
> dependencies when talking to the job API. Artifacts that need to be staged 
> include the user code itself, any SDK components not included in the 
> container image, and the list of Python packages that must be installed at 
> runtime.
>  
> Artifacts that are currently expected can be found in the harness boot code: 
> [https://github.com/apache/beam/blob/58e3b06bee7378d2d8db1c8dd534b415864f63e1/sdks/python/container/boot.go#L52.]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to