robertwb commented on a change in pull request #11708:
URL: https://github.com/apache/beam/pull/11708#discussion_r426061393



##########
File path: sdks/python/apache_beam/runners/portability/abstract_job_service.py
##########
@@ -313,39 +329,62 @@ def prepare(self):
     with tempfile.NamedTemporaryFile(suffix='.jar') as tout:
       self._jar = tout.name
     shutil.copy(self._executable_jar, self._jar)
-    with zipfile.ZipFile(self._jar, 'a', compression=zipfile.ZIP_DEFLATED) as 
z:
-      with z.open(self.PIPELINE_PATH, 'w') as fout:
-        fout.write(
-            json_format.MessageToJson(self._pipeline_proto).encode('utf-8'))
-      with z.open(self.PIPELINE_OPTIONS_PATH, 'w') as fout:
-        fout.write(
-            json_format.MessageToJson(self._pipeline_options).encode('utf-8'))
-      with z.open(self.PIPELINE_MANIFEST, 'w') as fout:
-        fout.write(
-            json.dumps({
-                'defaultJobName': self.PIPELINE_NAME
-            }).encode('utf-8'))
     self._start_artifact_service(self._jar, self._artifact_port)
 
   def _start_artifact_service(self, jar, requested_port):
-    self._artifact_staging_service = artifact_service.ZipFileArtifactService(
-        jar, self.ARTIFACT_FOLDER)
+    self._artifact_manager = JarArtifactManager(self._jar, 
self.ARTIFACT_FOLDER)
+    self._artifact_staging_service = artifact_service.ArtifactStagingService(
+        self._artifact_manager.file_writer)
+    self._artifact_staging_service.register_job(
+        self._job_id,
+        {
+            env_id: env.dependencies
+            for (env_id,
+                 env) in self._pipeline_proto.components.environments.items()
+        })
     self._artifact_staging_server = grpc.server(futures.ThreadPoolExecutor())
     port = self._artifact_staging_server.add_insecure_port(
         '[::]:%s' % requested_port)
-    
beam_artifact_api_pb2_grpc.add_LegacyArtifactStagingServiceServicer_to_server(
+    beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
         self._artifact_staging_service, self._artifact_staging_server)
     self._artifact_staging_endpoint = endpoints_pb2.ApiServiceDescriptor(
         url='localhost:%d' % port)
     self._artifact_staging_server.start()
     _LOGGER.info('Artifact server started on port %s', port)
+    _LOGGER.error('Artifact server started on port %s', port)

Review comment:
       Leftover debugging. Removed. 

##########
File path: sdks/python/apache_beam/runners/portability/abstract_job_service.py
##########
@@ -278,6 +279,23 @@ def to_runner_api(self):
         state=self.state)
 
 
+class JarArtifactManager(object):
+  def __init__(self, jar_path, root):
+    self._root = root
+    self._zipfile_handle = zipfile.ZipFile(jar_path, 'a')
+
+  def close(self):
+    self._zipfile_handle.close()
+
+  def file_writer(self, path):

Review comment:
       Done.

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/io/ClassLoaderFileSystem.java
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.fs.CreateOptions;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.io.fs.ResolveOptions;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.PipelineOptions;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/** A read-only {@link FileSystem} implementation looking up resources using a 
ClassLoader. */
+public class ClassLoaderFileSystem extends 
FileSystem<ClassLoaderFileSystem.ClassLoaderResourceId> {

Review comment:
       Yes, we'll want to do that too (and use the new artifact api). 

##########
File path: sdks/python/apache_beam/runners/portability/abstract_job_service.py
##########
@@ -313,39 +329,62 @@ def prepare(self):
     with tempfile.NamedTemporaryFile(suffix='.jar') as tout:
       self._jar = tout.name
     shutil.copy(self._executable_jar, self._jar)
-    with zipfile.ZipFile(self._jar, 'a', compression=zipfile.ZIP_DEFLATED) as 
z:
-      with z.open(self.PIPELINE_PATH, 'w') as fout:
-        fout.write(
-            json_format.MessageToJson(self._pipeline_proto).encode('utf-8'))
-      with z.open(self.PIPELINE_OPTIONS_PATH, 'w') as fout:
-        fout.write(
-            json_format.MessageToJson(self._pipeline_options).encode('utf-8'))
-      with z.open(self.PIPELINE_MANIFEST, 'w') as fout:
-        fout.write(
-            json.dumps({
-                'defaultJobName': self.PIPELINE_NAME
-            }).encode('utf-8'))
     self._start_artifact_service(self._jar, self._artifact_port)
 
   def _start_artifact_service(self, jar, requested_port):
-    self._artifact_staging_service = artifact_service.ZipFileArtifactService(
-        jar, self.ARTIFACT_FOLDER)
+    self._artifact_manager = JarArtifactManager(self._jar, 
self.ARTIFACT_FOLDER)
+    self._artifact_staging_service = artifact_service.ArtifactStagingService(
+        self._artifact_manager.file_writer)
+    self._artifact_staging_service.register_job(
+        self._job_id,
+        {
+            env_id: env.dependencies
+            for (env_id,
+                 env) in self._pipeline_proto.components.environments.items()
+        })
     self._artifact_staging_server = grpc.server(futures.ThreadPoolExecutor())
     port = self._artifact_staging_server.add_insecure_port(
         '[::]:%s' % requested_port)
-    
beam_artifact_api_pb2_grpc.add_LegacyArtifactStagingServiceServicer_to_server(
+    beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
         self._artifact_staging_service, self._artifact_staging_server)
     self._artifact_staging_endpoint = endpoints_pb2.ApiServiceDescriptor(
         url='localhost:%d' % port)
     self._artifact_staging_server.start()
     _LOGGER.info('Artifact server started on port %s', port)
+    _LOGGER.error('Artifact server started on port %s', port)
     return port
 
   def _stop_artifact_service(self):
     self._artifact_staging_server.stop(1)
-    self._artifact_staging_service.close()
-    self._artifact_manifest_location = (
-        self._artifact_staging_service.retrieval_token(self._job_id))
+    self._artifact_manifest_location = None

Review comment:
       Yep. 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


Reply via email to