You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by jk...@apache.org on 2017/06/13 20:21:21 UTC

[1/2] beam git commit: Improves message when transitively serializing PipelineOptions

Repository: beam
Updated Branches:
  refs/heads/master 7d0f24a0d -> 996e35c1d


Improves message when transitively serializing PipelineOptions


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3a48e47c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3a48e47c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3a48e47c

Branch: refs/heads/master
Commit: 3a48e47c73520580021c21a037ab412a761d0eeb
Parents: 7d0f24a
Author: Eugene Kirpichov <ki...@google.com>
Authored: Mon Jun 12 11:51:39 2017 -0700
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Tue Jun 13 13:04:13 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/beam/findbugs-filter.xml      |  9 +++++++++
 .../beam/sdk/options/ProxyInvocationHandler.java     | 15 ++++++++++++++-
 .../beam/sdk/options/ProxyInvocationHandlerTest.java | 12 ++++++++++++
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/3a48e47c/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
index 3430750..0c9080d 100644
--- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
@@ -412,4 +412,13 @@
     <!-- PTransforms do not actually support serialization. -->
   </Match>
 
+  <Match>
+    <Class name="org.apache.beam.sdk.options.ProxyInvocationHandler"/>
+    <Field name="~.*"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--
+      ProxyInvocationHandler implements Serializable only for the sake of throwing an informative
+      exception in writeObject()
+    -->
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/beam/blob/3a48e47c/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index eda21a8..3842388 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -45,6 +45,8 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.MutableClassToInstanceMap;
 import java.beans.PropertyDescriptor;
 import java.io.IOException;
+import java.io.NotSerializableException;
+import java.io.Serializable;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
@@ -87,7 +89,7 @@ import org.apache.beam.sdk.util.common.ReflectHelpers;
  * {@link PipelineOptions#as(Class)}.
  */
 @ThreadSafe
-class ProxyInvocationHandler implements InvocationHandler {
+class ProxyInvocationHandler implements InvocationHandler, Serializable {
   /**
    * No two instances of this class are considered equivalent hence we generate a random hash code.
    */
@@ -164,6 +166,17 @@ class ProxyInvocationHandler implements InvocationHandler {
         + Arrays.toString(args) + "].");
   }
 
+  private void writeObject(java.io.ObjectOutputStream stream)
+      throws IOException {
+    throw new NotSerializableException(
+        "PipelineOptions objects are not serializable and should not be embedded into transforms "
+            + "(did you capture a PipelineOptions object in a field or in an anonymous class?). "
+            + "Instead, if you're using a DoFn, access PipelineOptions at runtime "
+            + "via ProcessContext/StartBundleContext/FinishBundleContext.getPipelineOptions(), "
+            + "or pre-extract necessary fields from PipelineOptions "
+            + "at pipeline construction time.");
+  }
+
   /**
    * Track whether options values are explicitly set, or retrieved from defaults.
    */

http://git-wip-us.apache.org/repos/asf/beam/blob/3a48e47c/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index 2c43f57..d90cb42 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -44,6 +44,7 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import com.google.common.testing.EqualsTester;
 import java.io.IOException;
+import java.io.NotSerializableException;
 import java.io.Serializable;
 import java.util.HashSet;
 import java.util.List;
@@ -54,6 +55,7 @@ import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
@@ -1019,4 +1021,14 @@ public class ProxyInvocationHandlerTest {
     DisplayData data = DisplayData.from(options);
     assertThat(data, not(hasDisplayItem("value")));
   }
+
+  private static class CapturesOptions implements Serializable {
+    PipelineOptions options = PipelineOptionsFactory.create();
+  }
+
+  @Test
+  public void testOptionsAreNotSerializable() {
+    expectedException.expectCause(Matchers.<Throwable>instanceOf(NotSerializableException.class));
+    SerializableUtils.clone(new CapturesOptions());
+  }
 }


[2/2] beam git commit: This closes #3344: Improves message when transitively serializing PipelineOptions

Posted by jk...@apache.org.
This closes #3344: Improves message when transitively serializing PipelineOptions


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/996e35c1
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/996e35c1
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/996e35c1

Branch: refs/heads/master
Commit: 996e35c1de8310a265c890cf8ca8e92416b3d16b
Parents: 7d0f24a 3a48e47
Author: Eugene Kirpichov <ki...@google.com>
Authored: Tue Jun 13 13:04:28 2017 -0700
Committer: Eugene Kirpichov <ki...@google.com>
Committed: Tue Jun 13 13:04:28 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/beam/findbugs-filter.xml      |  9 +++++++++
 .../beam/sdk/options/ProxyInvocationHandler.java     | 15 ++++++++++++++-
 .../beam/sdk/options/ProxyInvocationHandlerTest.java | 12 ++++++++++++
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------