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

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

                Author: ASF GitHub Bot
            Created on: 10/May/18 02:35
            Start Date: 10/May/18 02:35
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on a change in pull request #5273: 
[BEAM-3883] Adding Client to push artifacts to artifact staging service
URL: https://github.com/apache/beam/pull/5273#discussion_r187189738
 
 

 ##########
 File path: 
sdks/python/apache_beam/runners/portability/artifact_service_client.py
 ##########
 @@ -0,0 +1,104 @@
+# 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.
+#
+"""A :class:`FileHandler` to work with :class:`ArtifactStagingServiceStub`.
+"""
+
+from __future__ import absolute_import
+from __future__ import division
+from __future__ import print_function
+
+import os
+
+from apache_beam.portability.api import beam_artifact_api_pb2
+from apache_beam.portability.api import beam_artifact_api_pb2_grpc
+from apache_beam.runners.portability.stager import FileHandler
+
+
+class ArtifactStagingFileHandler(FileHandler):
+  """:class:`FileHandler` to push files to ArtifactStagingService.
+
+  The class keeps track of pushed files and user is expected to call
+  :fun:`commit_manifest` once all files are uploaded.
+  Once :fun:`commit_manifest` is called, no further operations can be performed
+  on the class.
+
+  Note: This class is not thread safe and user of this class should ensure
+  thread safety.
+  """
+
+  def __init__(self, artifact_service_channel):
+    """Creates a new FileHandler to upload file to ArtifactStagingService.
+
+    Args:
+      artifact_service_channel: Channel used to interact with
+        ArtifactStagingService.User owns the channel and should close it when
+        finished.
+    """
+    super(ArtifactStagingFileHandler, self).__init__()
+    self._artifact_staging_stub = beam_artifact_api_pb2_grpc.\
+        ArtifactStagingServiceStub(channel=artifact_service_channel)
+    self._artifacts = []
+    self.closed = False
+
+  def file_copy(self, from_path, to_path):
+    """Uploads a file to ArtifactStagingService.
+
+    Note: Downloading/copying file from remote server is not supported.
+    Args:
+      from_path: Path of file to be uploaded.
+      to_path: File name on the artifact server.
+    """
+    self._check_closed()
+    if not os.path.isfile(from_path):
+      raise ValueError(
+          'Can only copy local file to artifact server. from_path: {0} '
+          'to_path: {1}'.format(from_path, to_path))
+
+    def artifact_request_generator():
+      metadata = beam_artifact_api_pb2.ArtifactMetadata(name=to_path)
+      request = beam_artifact_api_pb2.PutArtifactRequest(metadata=metadata)
+      yield request
+      with open(from_path, 'rb') as f:
+        while True:
+          chunk = f.read(2 << 12)  # 4kb
 
 Review comment:
   Sure, we can go till 2MB. GRPC has a payload size limit of 4MB so 2MB should 
be ok.

----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 100376)
    Time Spent: 5.5h  (was: 5h 20m)

> 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: 5.5h
>  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