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

[2/2] beam git commit: Remove DeterministicStandardCoder

Remove DeterministicStandardCoder

This isn't a particularly useful Coder. It has no defined methods other
than verifyDeterministic, which has an empty implementation.
Additionally, there are no guarantees that a DeterministicStandardCoder
is determinsitic.


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

Branch: refs/heads/master
Commit: e3b25215a2d91d363e3889c94a94ae6c0fc7b14d
Parents: cf5450f
Author: Thomas Groh <tg...@google.com>
Authored: Wed Apr 19 17:58:58 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Apr 21 16:24:14 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/coders/AtomicCoder.java |  5 ++-
 .../sdk/coders/DeterministicStandardCoder.java  | 39 --------------------
 .../beam/sdk/coders/CoderRegistryTest.java      | 18 +++------
 .../beam/sdk/coders/NullableCoderTest.java      |  5 ++-
 .../beam/sdk/util/SerializableUtilsTest.java    | 12 ++++--
 5 files changed, 22 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e3b25215/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
index c024f89..816af87 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
@@ -28,10 +28,13 @@ import java.util.List;
  *
  * @param <T> the type of the values being transcoded
  */
-public abstract class AtomicCoder<T> extends DeterministicStandardCoder<T> {
+public abstract class AtomicCoder<T> extends StandardCoder<T> {
   protected AtomicCoder() { }
 
   @Override
+  public void verifyDeterministic() throws NonDeterministicException { }
+
+  @Override
   public final List<Coder<?>> getCoderArguments() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/e3b25215/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
deleted file mode 100644
index 8998ea5..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DeterministicStandardCoder.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.coders;
-
-/**
- * A {@link DeterministicStandardCoder} is a {@link StandardCoder} that is
- * deterministic, in the sense that for objects considered equal
- * according to {@link Object#equals(Object)}, the encoded bytes are
- * also equal.
- *
- * @param <T> the type of the values being transcoded
- */
-public abstract class DeterministicStandardCoder<T> extends StandardCoder<T> {
-  protected DeterministicStandardCoder() {}
-
-  /**
-   * {@inheritDoc}
-   *
-   * @throws NonDeterministicException never, unless overridden. A
-   * {@link DeterministicStandardCoder} is presumed deterministic.
-   */
-  @Override
-  public void verifyDeterministic() throws NonDeterministicException { }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/e3b25215/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
index 774ca9d..10e011f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
@@ -46,7 +46,6 @@ import org.apache.beam.sdk.transforms.Create;
 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.util.CloudObject;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
@@ -115,7 +114,7 @@ public class CoderRegistryTest {
   }
 
   @SuppressWarnings("rawtypes") // this class exists to fail a test because of its rawtypes
-  private class MyListCoder extends DeterministicStandardCoder<List> {
+  private class MyListCoder extends StandardCoder<List> {
     @Override
     public void encode(List value, OutputStream outStream, Context context)
         throws CoderException, IOException {
@@ -131,6 +130,9 @@ public class CoderRegistryTest {
     public List<Coder<?>> getCoderArguments() {
       return Collections.emptyList();
     }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {}
   }
 
   @Test
@@ -454,7 +456,7 @@ public class CoderRegistryTest {
 
   private static class MyValue { }
 
-  private static class MyValueCoder implements Coder<MyValue> {
+  private static class MyValueCoder extends CustomCoder<MyValue> {
 
     private static final MyValueCoder INSTANCE = new MyValueCoder();
     private static final TypeDescriptor<MyValue> TYPE_DESCRIPTOR = TypeDescriptor.of(MyValue.class);
@@ -481,16 +483,6 @@ public class CoderRegistryTest {
     }
 
     @Override
-    public List<? extends Coder<?>> getCoderArguments() {
-      return null;
-    }
-
-    @Override
-    public CloudObject asCloudObject() {
-      return null;
-    }
-
-    @Override
     public void verifyDeterministic() { }
 
     @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/e3b25215/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
index 51e7bad..052144e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
@@ -177,7 +177,7 @@ public class NullableCoderTest {
     assertThat(TEST_CODER.getEncodedTypeDescriptor(), equalTo(TypeDescriptor.of(String.class)));
   }
 
-  private static class EntireStreamExpectingCoder extends DeterministicStandardCoder<String> {
+  private static class EntireStreamExpectingCoder extends StandardCoder<String> {
     @Override
     public void encode(
         String value, OutputStream outStream, Context context) throws IOException {
@@ -196,5 +196,8 @@ public class NullableCoderTest {
     public List<? extends Coder<?>> getCoderArguments() {
       return Collections.emptyList();
     }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {}
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/e3b25215/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
index 9f86ed2..e22f30e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
@@ -29,7 +29,7 @@ import java.io.Serializable;
 import java.util.List;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.DeterministicStandardCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -87,7 +87,7 @@ public class SerializableUtilsTest {
   }
 
   /** A {@link Coder} that is not serializable by Java. */
-  private static class UnserializableCoderByJava extends DeterministicStandardCoder<Object> {
+  private static class UnserializableCoderByJava extends StandardCoder<Object> {
     private final Object unserializableField = new Object();
 
     @Override
@@ -105,6 +105,9 @@ public class SerializableUtilsTest {
     public List<? extends Coder<?>> getCoderArguments() {
       return ImmutableList.of();
     }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {}
   }
 
   @Test
@@ -115,7 +118,7 @@ public class SerializableUtilsTest {
   }
 
   /** A {@link Coder} that is not serializable by Jackson. */
-  private static class UnserializableCoderByJackson extends DeterministicStandardCoder<Object> {
+  private static class UnserializableCoderByJackson extends StandardCoder<Object> {
     private final SerializableByJava unserializableField;
 
     public UnserializableCoderByJackson(SerializableByJava unserializableField) {
@@ -150,6 +153,9 @@ public class SerializableUtilsTest {
     public List<? extends Coder<?>> getCoderArguments() {
       return ImmutableList.of();
     }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {}
   }
 
   @Test