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

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

                Author: ASF GitHub Bot
            Created on: 24/Dec/19 00:28
            Start Date: 24/Dec/19 00:28
    Worklog Time Spent: 10m 
      Work Description: ibzib commented on pull request #10403: [BEAM-8891] 
Spark uber jar job server
URL: https://github.com/apache/beam/pull/10403#discussion_r361032469
 
 

 ##########
 File path: 
sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
 ##########
 @@ -0,0 +1,318 @@
+#
+# 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 job server submitting portable pipelines as uber jars to Spark."""
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import itertools
+import json
+import logging
+import shutil
+import tempfile
+import time
+import zipfile
+from concurrent import futures
+
+import grpc
+import requests
+from google.protobuf import json_format
+
+from apache_beam.options import pipeline_options
+from apache_beam.portability.api import beam_artifact_api_pb2_grpc
+from apache_beam.portability.api import beam_job_api_pb2
+from apache_beam.portability.api import endpoints_pb2
+from apache_beam.runners.portability import abstract_job_service
+from apache_beam.runners.portability import artifact_service
+from apache_beam.runners.portability import job_server
+from apache_beam.utils.timestamp import Timestamp
+
+_LOGGER = logging.getLogger(__name__)
+
+
+class SparkUberJarJobServer(abstract_job_service.AbstractJobServiceServicer):
+  """A Job server which submits a self-contained Jar to a Spark cluster.
+
+  The jar contains the Beam pipeline definition, dependencies, and
+  the pipeline artifacts.
+  """
+
+  def __init__(self, master_url, options):
+    super(SparkUberJarJobServer, self).__init__()
+    self._master_url = master_url
+    self._executable_jar = 
(options.view_as(pipeline_options.SparkRunnerOptions)
+                            .spark_job_server_jar)
+    self._artifact_port = (options.view_as(pipeline_options.JobServerOptions)
+                           .artifact_port)
+    self._temp_dir = tempfile.mkdtemp(prefix='apache-beam-spark')
+
+  def start(self):
+    return self
+
+  def stop(self):
+    pass
+
+  def executable_jar(self):
+    url = (self._executable_jar or
+           job_server.JavaJarJobServer.path_to_beam_jar(
+               'runners:spark:job-server:shadowJar'))
+    return job_server.JavaJarJobServer.local_jar(url)
+
+  def create_beam_job(self, job_id, job_name, pipeline, options):
+    return SparkBeamJob(
+        self._master_url,
+        self.executable_jar(),
+        job_id,
+        job_name,
+        pipeline,
+        options,
+        artifact_port=self._artifact_port)
+
+
+class SparkBeamJob(abstract_job_service.AbstractBeamJob):
+  """Runs a single Beam job on Spark by staging all contents into a Jar
+  and uploading it via the Spark Rest API.
+
+  Note that the Spark Rest API is not enabled by default. It must be enabled by
+  setting the configuration property spark.master.rest.enabled to true."""
+
+  # These must agree with those defined in PortablePipelineJarUtils.java.
+  PIPELINE_FOLDER = 'BEAM-PIPELINE'
+  PIPELINE_MANIFEST = PIPELINE_FOLDER + '/pipeline-manifest.json'
+
+  # We only stage a single pipeline in the jar.
+  PIPELINE_NAME = 'pipeline'
+  PIPELINE_PATH = '/'.join(
+      [PIPELINE_FOLDER, PIPELINE_NAME, "pipeline.json"])
+  PIPELINE_OPTIONS_PATH = '/'.join(
+      [PIPELINE_FOLDER, PIPELINE_NAME, 'pipeline-options.json'])
+  ARTIFACT_MANIFEST_PATH = '/'.join(
+      [PIPELINE_FOLDER, PIPELINE_NAME, 'artifact-manifest.json'])
+  ARTIFACT_FOLDER = '/'.join([PIPELINE_FOLDER, PIPELINE_NAME, 'artifacts'])
 
 Review comment:
   Done.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 362820)
    Time Spent: 20m  (was: 10m)

> Create and submit Spark portable jar in Python
> ----------------------------------------------
>
>                 Key: BEAM-8891
>                 URL: https://issues.apache.org/jira/browse/BEAM-8891
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-spark
>            Reporter: Kyle Weaver
>            Assignee: Kyle Weaver
>            Priority: Major
>              Labels: portability-spark
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Similar to FlinkUberJarJobServer: 
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py



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

Reply via email to