From d5fc12051133e5078144857eff20c59dafa27d5b Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 19 Apr 2017 17:58:58 -0700 Subject: [PATCH] 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. --- .../apache/beam/sdk/coders/AtomicCoder.java | 5 ++- .../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(-) delete mode 100644 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/AtomicCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java index c024f89da889..816af87458f0 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,9 +28,12 @@ * * @param the type of the values being transcoded */ -public abstract class AtomicCoder extends DeterministicStandardCoder { +public abstract class AtomicCoder extends StandardCoder { protected AtomicCoder() { } + @Override + public void verifyDeterministic() throws NonDeterministicException { } + @Override public final List> getCoderArguments() { return null; 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 8998ea576532..000000000000 --- 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 the type of the values being transcoded - */ -public abstract class DeterministicStandardCoder extends StandardCoder { - protected DeterministicStandardCoder() {} - - /** - * {@inheritDoc} - * - * @throws NonDeterministicException never, unless overridden. A - * {@link DeterministicStandardCoder} is presumed deterministic. - */ - @Override - public void verifyDeterministic() throws NonDeterministicException { } -} 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 10177e79b167..d37ab65ce2c5 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 @@ -43,7 +43,6 @@ 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 void testRegisterInstantiatedCoder() throws Exception { } @SuppressWarnings("rawtypes") // this class exists to fail a test because of its rawtypes - private class MyListCoder extends DeterministicStandardCoder { + private class MyListCoder extends StandardCoder { @Override public void encode(List value, OutputStream outStream, Context context) throws CoderException, IOException { @@ -131,6 +130,9 @@ public List decode(InputStream inStream, Context context) public List> getCoderArguments() { return Collections.emptyList(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} } @Test @@ -456,7 +458,7 @@ private static class MyGenericClass { } private static class MyValue { } - private static class MyValueCoder implements Coder { + private static class MyValueCoder extends CustomCoder { private static final MyValueCoder INSTANCE = new MyValueCoder(); private static final TypeDescriptor TYPE_DESCRIPTOR = TypeDescriptor.of(MyValue.class); @@ -482,16 +484,6 @@ public MyValue decode(InputStream inStream, Context context) return new MyValue(); } - @Override - public List> getCoderArguments() { - return null; - } - - @Override - public CloudObject asCloudObject() { - return null; - } - @Override public void verifyDeterministic() { } 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 51e7bad85a91..052144e30848 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 void testEncodedTypeDescriptor() throws Exception { assertThat(TEST_CODER.getEncodedTypeDescriptor(), equalTo(TypeDescriptor.of(String.class))); } - private static class EntireStreamExpectingCoder extends DeterministicStandardCoder { + private static class EntireStreamExpectingCoder extends StandardCoder { @Override public void encode( String value, OutputStream outStream, Context context) throws IOException { @@ -196,5 +196,8 @@ public String decode(InputStream inStream, Context context) public List> getCoderArguments() { return Collections.emptyList(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} } } 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 9f86ed227ce0..e22f30e2b87c 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.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 void testSerializationError() { } /** A {@link Coder} that is not serializable by Java. */ - private static class UnserializableCoderByJava extends DeterministicStandardCoder { + private static class UnserializableCoderByJava extends StandardCoder { private final Object unserializableField = new Object(); @Override @@ -105,6 +105,9 @@ public Object decode(InputStream inStream, Context context) public List> getCoderArguments() { return ImmutableList.of(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} } @Test @@ -115,7 +118,7 @@ public void testEnsureSerializableWithUnserializableCoderByJava() { } /** A {@link Coder} that is not serializable by Jackson. */ - private static class UnserializableCoderByJackson extends DeterministicStandardCoder { + private static class UnserializableCoderByJackson extends StandardCoder { private final SerializableByJava unserializableField; public UnserializableCoderByJackson(SerializableByJava unserializableField) { @@ -150,6 +153,9 @@ public Object decode(InputStream inStream, Context context) public List> getCoderArguments() { return ImmutableList.of(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} } @Test