You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/05/01 01:30:08 UTC

[GitHub] [beam] chamikaramj commented on a change in pull request #11557: [BEAM-9845] Stage artifacts over expansion service.

chamikaramj commented on a change in pull request #11557:
URL: https://github.com/apache/beam/pull/11557#discussion_r418376845



##########
File path: sdks/python/apache_beam/transforms/sql_test.py
##########
@@ -66,20 +66,8 @@ class SqlTransformTest(unittest.TestCase):
         --tests apache_beam.transforms.sql_test \\
         --test-pipeline-options="--runner=FlinkRunner"
   """
-  @staticmethod
-  def make_test_pipeline():
-    path_to_jar = subprocess_server.JavaJarServer.path_to_beam_jar(
-        ":sdks:java:extensions:sql:expansion-service:shadowJar")
-    test_pipeline = TestPipeline()
-    # TODO(BEAM-9238): Remove this when it's no longer needed for artifact
-    # staging.
-    test_pipeline.get_pipeline_options().view_as(DebugOptions).experiments = [
-        'jar_packages=' + path_to_jar

Review comment:
       Can we update 'validate_runner_xlang_test.py' for Spark and Flink as well or do we need additional changes before that.
   jar_packages cannot be completely removed yet since Dataflow needs it.

##########
File path: sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
##########
@@ -317,6 +321,14 @@ default InputT createInput(Pipeline p, Map<String, PCollection<?>> inputs) {
     Pipeline pipeline = Pipeline.create();
     ExperimentalOptions.addExperiment(
         pipeline.getOptions().as(ExperimentalOptions.class), "beam_fn_api");
+    List<String> classpathResources =
+        detectClassPathResourcesToStage(Environments.class.getClassLoader(), pipeline.getOptions());

Review comment:
       This assumes that the set of classpath resources required to execute Java SDK harness and execute the cross-langauge Java step to be the same as the set of classpath resources specified when starting up the expansion service, right ?
   
   I'm wondering if we can always rely on this. At least we should document this so that users are aware how to stage additional resources if needed.




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