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 2022/03/11 00:38:07 UTC

[GitHub] [beam] ihji commented on a change in pull request #17035: [BEAM-14038] Auto-startup for Python expansion service.

ihji commented on a change in pull request #17035:
URL: https://github.com/apache/beam/pull/17035#discussion_r824282843



##########
File path: sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/ExternalPythonTransform.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.extensions.python;
+
+import java.util.Set;
+import java.util.UUID;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
+import org.apache.beam.runners.core.construction.External;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+
+/** Wrapper for invoking external Python transforms. */
+public class ExternalPythonTransform<InputT extends PInput, OutputT extends POutput>
+    extends PTransform<InputT, OutputT> {
+  private final String fullyQualifiedName;
+  private final Row args;
+  private final Row kwargs;
+
+  public ExternalPythonTransform(String fullyQualifiedName, Row args, Row kwargs) {
+    this.fullyQualifiedName = fullyQualifiedName;
+    this.args = args;
+    this.kwargs = kwargs;
+  }
+
+  @Override
+  public OutputT expand(InputT input) {
+    int port;
+    try {
+      port = PythonService.findAvailablePort();
+      PythonService service =
+          new PythonService(
+              "apache_beam.runners.portability.expansion_service_main",
+              "--port",
+              "" + port,
+              "--fully_qualified_name_glob",
+              "*");
+      Schema payloadSchema =
+          Schema.of(
+              Schema.Field.of("constructor", Schema.FieldType.STRING),
+              Schema.Field.of("args", Schema.FieldType.row(args.getSchema())),
+              Schema.Field.of("kwargs", Schema.FieldType.row(kwargs.getSchema())));
+      payloadSchema.setUUID(UUID.randomUUID());
+      Row payloadRow =
+          Row.withSchema(payloadSchema).addValues(fullyQualifiedName, args, kwargs).build();
+      ExternalTransforms.ExternalConfigurationPayload payload =
+          ExternalTransforms.ExternalConfigurationPayload.newBuilder()
+              .setSchema(SchemaTranslation.schemaToProto(payloadSchema, true))
+              .setPayload(
+                  ByteString.copyFrom(
+                      CoderUtils.encodeToByteArray(RowCoder.of(payloadSchema), payloadRow)))
+              .build();
+      try (AutoCloseable p = service.start()) {
+        PythonService.waitForPort("localhost", port, 5000);
+        PTransform<PInput, PCollectionTuple> transform =
+            External.<PInput, Object>of(
+                    "beam:transforms:python:fully_qualified_named",
+                    payload.toByteArray(),
+                    "localhost:" + port)
+                .withMultiOutputs();
+        PCollectionTuple outputs;
+        if (input instanceof PCollection) {
+          outputs = (PCollectionTuple) ((PCollection) input).apply(transform);

Review comment:
       Do we need this explicit casting to `(PCollectionTuple)` ?

##########
File path: sdks/java/extensions/python/src/main/resources/org/apache/beam/sdk/extensions/python/bootstrap_beam_venv.py
##########
@@ -0,0 +1,96 @@
+# 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 utility for bootstrapping a BeamPython install.
+
+This utility can be called with any version of Python, and attempts to create
+a Python virtual environment with the requested version of Beam, and any
+extra dependencies as required, installed.
+
+The virtual environment will be placed in Apache Beam's cache directory, and
+will be re-used if the parameters match.
+
+If this script exits successfully, the last line will be the full path to a
+suitable python executable.
+"""
+
+import argparse
+import hashlib
+import os
+import shutil
+import subprocess
+import sys
+
+
+def main():
+    if sys.version_info < (3, ):
+        # Run this script with Python 3.
+        os.execlp('python3', 'python3', *sys.argv)
+        return  # In case windows returns...
+
+    parser = argparse.ArgumentParser()
+    parser.add_argument('--python_version', help="Python major version.")
+    parser.add_argument('--beam_version',
+                        help="Beam version.",
+                        default="2.36.0")

Review comment:
       Is there any way to avoid hard-coded Beam version? Do we need to update this version string every time we release a new version of Beam?

##########
File path: sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.extensions.python;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.common.base.Charsets;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.common.io.ByteStreams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Utility to bootstrap and start a Beam Python service. */
+public class PythonService {
+  private static final Logger LOG = LoggerFactory.getLogger(PythonService.class);
+
+  private final String module;
+  private final List<String> args;
+
+  public PythonService(String module, List<String> args) {
+    this.module = module;
+    this.args = args;
+  }
+
+  public PythonService(String module, String... args) {
+    this(module, Arrays.asList(args));
+  }
+
+  public AutoCloseable start() throws IOException, InterruptedException {
+    File bootstrapScript = File.createTempFile("bootstrap_beam_venv", ".py");
+    bootstrapScript.deleteOnExit();
+    try (FileOutputStream fout = new FileOutputStream(bootstrapScript.getAbsolutePath())) {
+      ByteStreams.copy(getClass().getResourceAsStream("bootstrap_beam_venv.py"), fout);
+    }
+    List<String> bootstrapCommand = ImmutableList.of("python", bootstrapScript.getAbsolutePath());
+    LOG.info("Running bootstrap command " + bootstrapCommand);
+    Process bootstrap =
+        new ProcessBuilder("python", bootstrapScript.getAbsolutePath())

Review comment:
       Isn't it simpler to run `bootstrap_beam_venv.py` not by creating a temporary copy but just passing the script text resource to python executable via stdin pipe?

##########
File path: sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/ExternalPythonTransform.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.extensions.python;
+
+import java.util.Set;
+import java.util.UUID;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
+import org.apache.beam.runners.core.construction.External;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+
+/** Wrapper for invoking external Python transforms. */
+public class ExternalPythonTransform<InputT extends PInput, OutputT extends POutput>
+    extends PTransform<InputT, OutputT> {
+  private final String fullyQualifiedName;
+  private final Row args;
+  private final Row kwargs;
+
+  public ExternalPythonTransform(String fullyQualifiedName, Row args, Row kwargs) {
+    this.fullyQualifiedName = fullyQualifiedName;
+    this.args = args;
+    this.kwargs = kwargs;
+  }
+
+  @Override
+  public OutputT expand(InputT input) {
+    int port;
+    try {
+      port = PythonService.findAvailablePort();
+      PythonService service =
+          new PythonService(
+              "apache_beam.runners.portability.expansion_service_main",
+              "--port",
+              "" + port,
+              "--fully_qualified_name_glob",
+              "*");
+      Schema payloadSchema =
+          Schema.of(
+              Schema.Field.of("constructor", Schema.FieldType.STRING),
+              Schema.Field.of("args", Schema.FieldType.row(args.getSchema())),
+              Schema.Field.of("kwargs", Schema.FieldType.row(kwargs.getSchema())));
+      payloadSchema.setUUID(UUID.randomUUID());

Review comment:
       Just out of curiosity, why do we need to set UUID for the schema?




-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org