You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/11/09 10:28:00 UTC

[jira] [Work logged] (BEAM-3921) Scripting extension based on Java Scripting API (JSR-223)

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

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

                Author: ASF GitHub Bot
            Created on: 09/Nov/18 10:27
            Start Date: 09/Nov/18 10:27
    Worklog Time Spent: 10m 
      Work Description: iemejia closed pull request #4944:  [BEAM-3921] Scripting extension based on Java Scripting API (JSR-223)
URL: https://github.com/apache/beam/pull/4944
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pom.xml b/pom.xml
index fa925131441..2f7c98faff1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -532,6 +532,12 @@
         <version>${project.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-extensions-scripting</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-extensions-sketching</artifactId>
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index dbffdd640ad..5192f145d03 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -36,6 +36,7 @@
     <module>jackson</module>
     <module>join-library</module>
     <module>protobuf</module>
+    <module>scripting</module>
     <module>sketching</module>
     <module>sorter</module>
     <module>sql</module>
diff --git a/sdks/java/extensions/scripting/build.gradle b/sdks/java/extensions/scripting/build.gradle
new file mode 100644
index 00000000000..a1dc966c3a0
--- /dev/null
+++ b/sdks/java/extensions/scripting/build.gradle
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+apply from: project(":").file("build_rules.gradle")
+applyJavaNature()
+
+description = "Apache Beam :: SDKs :: Java :: Extensions :: Scripting"
+
+dependencies {
+  shadow project(path: ":beam-sdks-java-core", configuration: "shadow")
+  compile library.java.guava
+  shadowTest project(path: ":beam-runners-direct-java", configuration: "shadow")
+  shadowTest library.java.hamcrest_core
+  shadowTest library.java.hamcrest_library
+  shadowTest library.java.junit
+}
diff --git a/sdks/java/extensions/scripting/pom.xml b/sdks/java/extensions/scripting/pom.xml
new file mode 100644
index 00000000000..ed3b28b1cb8
--- /dev/null
+++ b/sdks/java/extensions/scripting/pom.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-extensions-parent</artifactId>
+    <version>2.5.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-extensions-scripting</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Extensions :: Scripting</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <!-- Dependency for tests -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDo.java b/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDo.java
new file mode 100644
index 00000000000..9fcac5a2aec
--- /dev/null
+++ b/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDo.java
@@ -0,0 +1,189 @@
+/*
+ * 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.scripting;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+import javax.script.SimpleScriptContext;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * A transform to write simple ParDo transformations with a scripting language
+ * supported by Java JSR-223.
+ */
+@Experimental
+public abstract class ScriptingParDo<InputT, OutputT>
+    extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
+
+  private ValueProvider<String> language = StaticValueProvider.of("js");
+  private ValueProvider<String> script = StaticValueProvider.of("");
+  private Coder<OutputT> coder;
+
+  public static <InputT, OutputT> ScriptingParDo<InputT, OutputT> of(final Coder<OutputT> coder) {
+    final ScriptingParDo<InputT, OutputT> scripting = new ScriptingParDo<InputT, OutputT>() {};
+    scripting.coder = coder;
+    return scripting;
+  }
+
+  public ScriptingParDo<InputT, OutputT> withLanguage(final String language) {
+    checkArgument(language != null, "language can not be null");
+    return withLanguage(ValueProvider.StaticValueProvider.of(language));
+  }
+
+  public ScriptingParDo<InputT, OutputT> withLanguage(final ValueProvider<String> language) {
+    checkArgument(language != null, "language can not be null");
+    this.language = language;
+    return this;
+  }
+
+  public ScriptingParDo<InputT, OutputT> withScript(final String script) {
+    checkArgument(script != null, "script can not be null");
+    return withScript(ValueProvider.StaticValueProvider.of(script));
+  }
+
+  public ScriptingParDo<InputT, OutputT> withScript(final ValueProvider<String> script) {
+    checkArgument(script != null, "script can not be null");
+    this.script = script;
+    return this;
+  }
+
+  @Override
+  public PCollection<OutputT> expand(final PCollection<InputT> apCollection) {
+    if (language == null || script == null || script.get().isEmpty()) {
+      throw new IllegalArgumentException("Language and Script must be set");
+    }
+    return apCollection.apply(ParDo.of(new ScriptingDoFn<>(language.get(), script.get())));
+  }
+
+  @Override // ensure we don't always need to set the coder
+  public <T> Coder<T> getDefaultOutputCoder(
+      final PCollection<InputT> input, final PCollection<T> output)
+      throws CannotProvideCoderException {
+    if (coder != null) {
+      return (Coder<T>) coder;
+    }
+    final Type superclass = getClass().getGenericSuperclass();
+    if (ParameterizedType.class.isInstance(superclass)) {
+      final Type type = ParameterizedType.class.cast(superclass).getActualTypeArguments()[1];
+      return (Coder<T>) output.getPipeline().getCoderRegistry().getCoder(TypeDescriptor.of(type));
+    }
+    return (Coder<T>) SerializableCoder.of(Serializable.class);
+  }
+
+  private static class ScriptingDoFn<InputT, OutputT> extends DoFn<InputT, OutputT> {
+    private String language;
+    private String script;
+
+    private volatile ScriptEngine engine;
+    private CompiledScript compiledScript;
+
+    ScriptingDoFn(final String language, final String script) {
+      this.language = language;
+      this.script = script;
+    }
+
+    @Setup
+    public void setup() {
+      final ScriptEngineManager manager =
+        new ScriptEngineManager(Thread.currentThread().getContextClassLoader());
+      engine = manager.getEngineByExtension(language);
+      if (engine == null) {
+        engine = manager.getEngineByName(language);
+        if (engine == null) {
+          engine = manager.getEngineByMimeType(language);
+        }
+      }
+      if (engine == null) {
+        throw new IllegalArgumentException("Language [" + language + "] "
+                + "not supported. Check that the needed depencencies are configured.");
+      }
+
+      if (Compilable.class.isInstance(engine)) {
+        try {
+          compiledScript = Compilable.class.cast(engine).compile(script);
+        } catch (ScriptException e) {
+          throw new IllegalStateException(e);
+        }
+      } else {
+        compiledScript = new CompiledScript() {
+          @Override
+          public Object eval(final ScriptContext context) throws ScriptException {
+            return engine.eval(script, context);
+          }
+
+          @Override
+          public ScriptEngine getEngine() {
+            return engine;
+          }
+        };
+      }
+    }
+
+    @ProcessElement
+    public void processElement(final ProcessContext context) {
+      final Bindings bindings = engine.createBindings();
+      bindings.put("context", context);
+
+      final SimpleScriptContext scriptContext = new SimpleScriptContext();
+      scriptContext.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
+
+      try {
+        final Object eval = compiledScript.eval(scriptContext);
+        if (eval != null) {
+          // if the script returns a value we put it in the context otherwise we asume the script
+          // already did the output via context.output(...)
+          context.output((OutputT) eval);
+        }
+      } catch (final ScriptException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    @Teardown
+    public void tearDown() {
+      //TODO compiledScript != null
+      if (AutoCloseable.class.isInstance(compiledScript)) {
+        try {
+          AutoCloseable.class.cast(compiledScript).close();
+        } catch (final Exception e) {
+          throw new IllegalStateException(e);
+        }
+      }
+    }
+  }
+}
diff --git a/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/package-info.java b/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/package-info.java
new file mode 100644
index 00000000000..01c4a2b9988
--- /dev/null
+++ b/sdks/java/extensions/scripting/src/main/java/org/apache/beam/sdk/extensions/scripting/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Transforms to support different languages using
+ * JSR-223 (Java Scripting for the Java Platform).
+ */
+package org.apache.beam.sdk.extensions.scripting;
diff --git a/sdks/java/extensions/scripting/src/test/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDoTest.java b/sdks/java/extensions/scripting/src/test/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDoTest.java
new file mode 100644
index 00000000000..911c37be8ae
--- /dev/null
+++ b/sdks/java/extensions/scripting/src/test/java/org/apache/beam/sdk/extensions/scripting/ScriptingParDoTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.scripting;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Tests for the ScriptingParDo transform. */
+public class ScriptingParDoTest {
+  @Rule public volatile TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testScriptingWithJs() {
+    final PCollection<Integer> out =
+        pipeline
+            .apply(Create.of("v1", "v22"))
+            .apply(
+                new ScriptingParDo<String, Integer>() {}.withLanguage("js")
+                    .withScript("context.output(context.element().length());"));
+    PAssert.that(out).containsInAnyOrder(2, 3);
+    final PipelineResult result = pipeline.run();
+    assertEquals(PipelineResult.State.DONE, result.waitUntilFinish());
+  }
+}
diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml
index ffd15464353..b425cbfcf07 100644
--- a/sdks/java/javadoc/pom.xml
+++ b/sdks/java/javadoc/pom.xml
@@ -107,6 +107,11 @@
       <artifactId>beam-sdks-java-extensions-protobuf</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-scripting</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-extensions-sketching</artifactId>
diff --git a/settings.gradle b/settings.gradle
index f0703a32adb..b9be8b0d5e3 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -80,6 +80,8 @@ include "beam-sdks-java-extensions-join-library"
 project(":beam-sdks-java-extensions-join-library").dir = file("sdks/java/extensions/join-library")
 include "beam-sdks-java-extensions-protobuf"
 project(":beam-sdks-java-extensions-protobuf").dir = file("sdks/java/extensions/protobuf")
+include "beam-sdks-java-extensions-scripting"
+project(":beam-sdks-java-extensions-scripting").dir = file("sdks/java/extensions/scripting")
 include "beam-sdks-java-extensions-sketching"
 project(":beam-sdks-java-extensions-sketching").dir = file("sdks/java/extensions/sketching")
 include "beam-sdks-java-extensions-sorter"


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 164312)
    Time Spent: 2.5h  (was: 2h 20m)

> Scripting extension based on Java Scripting API (JSR-223)
> ---------------------------------------------------------
>
>                 Key: BEAM-3921
>                 URL: https://issues.apache.org/jira/browse/BEAM-3921
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-ideas
>    Affects Versions: 2.5.0
>            Reporter: Ismaël Mejía
>            Assignee: Ismaël Mejía
>            Priority: Minor
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> An extension with transforms that package the Java Scripting API (JSR-223) [1] to allow users to specialize some transforms via a scripting language. It supports ValueProviders so users can template their scripts also in Dataflow.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)