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/07/08 21:50:56 UTC

[GitHub] [beam] robertwb commented on a diff in pull request #21809: [BEAM-14506] Adding testcases and examples for xlang Python RunInference

robertwb commented on code in PR #21809:
URL: https://github.com/apache/beam/pull/21809#discussion_r917155398


##########
sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.transforms;
+
+import java.util.Map;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.python.PythonExternalTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PythonCallableSource;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Wrapper for invoking external Python RunInference. */
+public class RunInference extends PTransform<PCollection<?>, PCollection<Row>> {
+  private final String modelLoader;
+  private final Schema schema;
+  private final Map<String, Object> kwargs;
+  private final String expansionService;
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.

Review Comment:
   s/lambda function/callable/ (and elsewhere)



##########
sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.transforms;
+
+import java.util.Map;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.python.PythonExternalTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PythonCallableSource;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Wrapper for invoking external Python RunInference. */
+public class RunInference extends PTransform<PCollection<?>, PCollection<Row>> {
+  private final String modelLoader;
+  private final Schema schema;
+  private final Map<String, Object> kwargs;
+  private final String expansionService;
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param exampleType A schema field type for the example column in output rows.
+   * @param inferenceType A schema field type for the inference column in output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(
+      String modelLoader, Schema.FieldType exampleType, Schema.FieldType inferenceType) {
+    Schema schema =
+        Schema.of(
+            Schema.Field.of("example", exampleType), Schema.Field.of("inference", inferenceType));
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param schema A schema for output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(String modelLoader, Schema schema) {
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Sets keyword arguments for RunInference constructor.
+   *
+   * @return A {@link RunInference} with keyword arguments.
+   */
+  public RunInference withKwarg(String key, Object arg) {
+    ImmutableMap.Builder<String, Object> builder =
+        ImmutableMap.<String, Object>builder().putAll(kwargs).put(key, arg);
+    return new RunInference(modelLoader, schema, builder.build(), expansionService);
+  }
+
+  /**
+   * Sets an expansion service endpoint for RunInference.
+   *
+   * @param expansionService A URL for a Python expansion service.
+   * @return A {@link RunInference} for the given expansion service endpoint.
+   */
+  public RunInference withExpansionService(String expansionService) {
+    return new RunInference(modelLoader, schema, kwargs, expansionService);
+  }
+
+  private RunInference(
+      String modelLoader, Schema schema, Map<String, Object> kwargs, String expansionService) {
+    this.modelLoader = modelLoader;
+    this.schema = schema;
+    this.kwargs = kwargs;
+    this.expansionService = expansionService;
+  }
+
+  @Override
+  public PCollection<Row> expand(PCollection<?> input) {
+    return input.apply(
+        PythonExternalTransform.<PCollection<?>, PCollection<Row>>from(
+                "apache_beam.ml.inference.base.RunInference.create", expansionService)

Review Comment:
   Generally "create" sounds like it's a synonym for the constructor. It should probably be renamed "from_callable" or something meaningful like that. 



##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -2309,8 +2311,10 @@ class BeamModulePlugin implements Plugin<Project> {
         throw new GradleException("unsupported java version.")
       }
       def setupTask = project.tasks.register(config.name+"Setup", Exec) {
-        dependsOn ':sdks:java:container:'+javaContainerSuffix+':docker'
-        dependsOn ':sdks:python:container:py'+pythonContainerSuffix+':docker'
+        if (!project.hasProperty('skipContainerBuilds')) {

Review Comment:
   When would this be desirable? Seems dangerous to make tests non-hermetic. 



##########
sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.transforms;
+
+import java.util.Map;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.python.PythonExternalTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PythonCallableSource;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Wrapper for invoking external Python RunInference. */
+public class RunInference extends PTransform<PCollection<?>, PCollection<Row>> {
+  private final String modelLoader;
+  private final Schema schema;
+  private final Map<String, Object> kwargs;
+  private final String expansionService;
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param exampleType A schema field type for the example column in output rows.
+   * @param inferenceType A schema field type for the inference column in output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(
+      String modelLoader, Schema.FieldType exampleType, Schema.FieldType inferenceType) {
+    Schema schema =
+        Schema.of(
+            Schema.Field.of("example", exampleType), Schema.Field.of("inference", inferenceType));
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param schema A schema for output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(String modelLoader, Schema schema) {
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Sets keyword arguments for RunInference constructor.

Review Comment:
   Sets keyword arguments for the model loader. 



##########
sdks/java/extensions/python/src/test/java/org/apache/beam/sdk/extensions/python/transforms/RunInferenceTransformTest.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.transforms;
+
+import java.util.Arrays;
+import java.util.Optional;
+import org.apache.beam.runners.core.construction.BaseExternalTest;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.UsesPythonExpansionService;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class RunInferenceTransformTest extends BaseExternalTest {
+  @Test
+  @Category({ValidatesRunner.class, UsesPythonExpansionService.class})
+  public void testRunInference() {
+    String stagingLocation =
+        Optional.ofNullable(System.getProperty("semiPersistDir")).orElse("/tmp");
+    Schema schema =
+        Schema.of(
+            Schema.Field.of("example", Schema.FieldType.array(Schema.FieldType.INT64)),
+            Schema.Field.of("inference", Schema.FieldType.INT32));
+    Row row0 = Row.withSchema(schema).addArray(0L, 0L).addValue(0).build();
+    Row row1 = Row.withSchema(schema).addArray(1L, 1L).addValue(1).build();
+    PCollection<Row> col =
+        testPipeline
+            .apply(Create.<Iterable<Long>>of(Arrays.asList(0L, 0L), Arrays.asList(1L, 1L)))
+            .setCoder(IterableCoder.of(VarLongCoder.of()))
+            .apply(
+                RunInference.of(
+                        "apache_beam.ml.inference.sklearn_inference.SklearnModelHandlerNumpy",
+                        schema)
+                    .withKwarg(
+                        "model_uri", String.format("%s/staged/sklearn_model", stagingLocation))

Review Comment:
   Where does the "staged" come from? I think we need some comments here about who creates the model and how it gets saved for this test. 



##########
sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/transforms/RunInference.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.transforms;
+
+import java.util.Map;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.python.PythonExternalTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PythonCallableSource;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Wrapper for invoking external Python RunInference. */
+public class RunInference extends PTransform<PCollection<?>, PCollection<Row>> {
+  private final String modelLoader;
+  private final Schema schema;
+  private final Map<String, Object> kwargs;
+  private final String expansionService;
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param exampleType A schema field type for the example column in output rows.
+   * @param inferenceType A schema field type for the inference column in output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(
+      String modelLoader, Schema.FieldType exampleType, Schema.FieldType inferenceType) {
+    Schema schema =
+        Schema.of(
+            Schema.Field.of("example", exampleType), Schema.Field.of("inference", inferenceType));
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Instantiates a multi-language wrapper for a Python RunInference with a given model loader.
+   *
+   * @param modelLoader A Python lambda function for a model loader class object.
+   * @param schema A schema for output rows.
+   * @return A {@link RunInference} for the given model loader.
+   */
+  public static RunInference of(String modelLoader, Schema schema) {
+    return new RunInference(modelLoader, schema, ImmutableMap.of(), "");
+  }
+
+  /**
+   * Sets keyword arguments for RunInference constructor.
+   *
+   * @return A {@link RunInference} with keyword arguments.
+   */
+  public RunInference withKwarg(String key, Object arg) {
+    ImmutableMap.Builder<String, Object> builder =
+        ImmutableMap.<String, Object>builder().putAll(kwargs).put(key, arg);
+    return new RunInference(modelLoader, schema, builder.build(), expansionService);
+  }
+
+  /**
+   * Sets an expansion service endpoint for RunInference.
+   *
+   * @param expansionService A URL for a Python expansion service.
+   * @return A {@link RunInference} for the given expansion service endpoint.
+   */
+  public RunInference withExpansionService(String expansionService) {
+    return new RunInference(modelLoader, schema, kwargs, expansionService);
+  }
+
+  private RunInference(
+      String modelLoader, Schema schema, Map<String, Object> kwargs, String expansionService) {
+    this.modelLoader = modelLoader;
+    this.schema = schema;
+    this.kwargs = kwargs;
+    this.expansionService = expansionService;
+  }
+
+  @Override
+  public PCollection<Row> expand(PCollection<?> input) {
+    return input.apply(
+        PythonExternalTransform.<PCollection<?>, PCollection<Row>>from(
+                "apache_beam.ml.inference.base.RunInference.create", expansionService)
+            .withKwarg("model_handler_provider", PythonCallableSource.of(modelLoader))
+            .withKwargs(kwargs)
+            .withOutputCoder(RowCoder.of(schema)));

Review Comment:
   I agree this is unfortunate, but there are lots of cases where the type inferencer can't figure this out, and it doesn't matter (much) in Python as Object is just fine (e.g. it gets fused away or we can use FastPrimitives/PickleCoder), but for cross language this information must be provided explicitly. 



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