diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index 8b823bc8fd17a..fbfe49aaa5ad6 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -624,11 +624,6 @@ public void verifyDeterministic() { public boolean consistentWithEquals() { return true; } - - @Override - public String getEncodingId() { - return ""; - } } /** @@ -694,11 +689,6 @@ public long getEncodedElementByteSize(KeyPrefix value, Coder.Context context) return VarInt.getLength(value.getSharedKeySize()) + VarInt.getLength(value.getUnsharedKeySize()); } - - @Override - public String getEncodingId() { - return ""; - } } /** @@ -779,10 +769,5 @@ public long getEncodedElementByteSize(Footer value, Coder.Context context) throws Exception { return Footer.FIXED_LENGTH; } - - @Override - public String getEncodingId() { - return ""; - } } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java index 4b07ca2d35aa9..66548e25b55a9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java @@ -113,11 +113,6 @@ protected long getEncodedElementByteSize(RandomAccessData value, Coder.Context c } return size + value.size; } - - @Override - public String getEncodingId() { - return ""; - } } public static final UnsignedLexicographicalComparator UNSIGNED_LEXICOGRAPHICAL_COMPARATOR = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java index d60a2cade7811..1d7cce57d3042 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java @@ -239,39 +239,6 @@ public DatumWriter initialValue() { }; } - /** - * The encoding identifier is designed to support evolution as per the design of Avro - * In order to use this class effectively, carefully read the Avro - * documentation at - * Schema Resolution - * to ensure that the old and new schema match. - * - *

In particular, this encoding identifier is guaranteed to be the same for {@code AvroCoder} - * instances of the same principal class, and otherwise distinct. The schema is not included - * in the identifier. - * - *

When modifying a class to be encoded as Avro, here are some guidelines; see the above link - * for greater detail. - * - *

- * - *

Code consuming this message class should be prepared to support all versions of - * the class until it is certain that no remaining serialized instances exist. - * - *

If backwards incompatible changes must be made, the best recourse is to change the name - * of your class. - */ - @Override - public String getEncodingId() { - return type.getName(); - } - /** * Returns the type this coder encodes/decodes. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java index 90ace9972b645..d628203a48c2c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java @@ -79,11 +79,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java index 8f45a56f8e2fd..81c5e9471ad65 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java @@ -76,11 +76,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java index 2d476977f03b4..173e91020427d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java @@ -78,11 +78,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java index 40331b8bd7ae0..a739da78aa171 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java @@ -69,11 +69,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java index 2ef166b700aa8..1a1be641169a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java @@ -85,11 +85,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 28d87e3b22a74..632cf8900661a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -27,7 +27,6 @@ import java.io.OutputStream; import java.io.Serializable; import java.util.Arrays; -import java.util.Collection; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; @@ -55,9 +54,6 @@ * inferred. See {@link KvCoder} for an example of a simple compound {@link Coder} that supports * automatic composition in the {@link CoderRegistry}. * - *

The binary format of a {@link Coder} is identified by {@link #getEncodingId()}; be sure to - * understand the requirements for evolving coder formats. - * *

All methods of a {@link Coder} are required to be thread safe. * * @param the type of the values being transcoded @@ -225,36 +221,6 @@ void registerByteSizeObserver( T value, ElementByteSizeObserver observer, Context context) throws Exception; - /** - * An identifier for the binary format written by {@link #encode}. - * - *

This value, along with the fully qualified class name, forms an identifier for the - * binary format of this coder. Whenever this value changes, the new encoding is considered - * incompatible with the prior format: It is presumed that the prior version of the coder will - * be unable to correctly read the new format and the new version of the coder will be unable to - * correctly read the old format. - * - *

If the format is changed in a backwards-compatible way (the Coder can still accept data from - * the prior format), such as by adding optional fields to a Protocol Buffer or Avro definition, - * and you want a Beam runner to understand that the new coder is compatible with the prior coder, - * this value must remain unchanged. It is then the responsibility of {@link #decode} to correctly - * read data from the prior format. - */ - @Experimental(Kind.CODER_ENCODING_ID) - String getEncodingId(); - - /** - * A collection of encodings supported by {@link #decode} in addition to the encoding - * from {@link #getEncodingId()} (which is assumed supported). - * - *

This information is not currently used for any purpose. It is descriptive only, - * and this method is subject to change. - * - * @see #getEncodingId() - */ - @Experimental(Kind.CODER_ENCODING_ID) - Collection getAllowedEncodings(); - /** * Returns the {@link TypeDescriptor} for the type encoded. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java index 2262e131a9f69..55ec2aacca316 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java @@ -33,8 +33,7 @@ * serialization. * *

To complete an implementation, subclasses must implement {@link Coder#encode} - * and {@link Coder#decode} methods. Anonymous subclasses must furthermore override - * {@link #getEncodingId}. + * and {@link Coder#decode} methods. * *

Not to be confused with {@link SerializableCoder} that encodes objects that implement the * {@link Serializable} interface. @@ -43,6 +42,7 @@ */ public abstract class CustomCoder extends StandardCoder implements Serializable { + @JsonCreator @Deprecated public static CustomCoder of( @@ -114,27 +114,6 @@ public void verifyDeterministic() throws NonDeterministicException { + " or they are presumed nondeterministic."); } - /** - * {@inheritDoc} - * - * @return The canonical class name for this coder. For stable data formats that are independent - * of class name, it is recommended to override this method. - * - * @throws UnsupportedOperationException when an anonymous class is used, since they do not have - * a stable canonical class name. - */ - @Override - public String getEncodingId() { - if (getClass().isAnonymousClass()) { - throw new UnsupportedOperationException( - String.format("Anonymous CustomCoder subclass %s must override getEncodingId()." - + " Otherwise, convert to a named class and getEncodingId() will be automatically" - + " generated from the fully qualified class name.", - getClass())); - } - return getClass().getCanonicalName(); - } - // This coder inherits isRegisterByteSizeObserverCheap, // getEncodedElementByteSize and registerByteSizeObserver // from StandardCoder. Override if we can do better. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java index 7e1154a584893..de9659b56e9a5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java @@ -19,13 +19,10 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Objects; -import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import java.util.Collection; -import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.values.TypeDescriptor; @@ -142,34 +139,6 @@ public String toString() { .toString(); } - /** - * {@inheritDoc} - * - * @return a {@link String} composed from the underlying coder class name and its encoding id. - * Note that this omits any description of the coding functions. These should be modified - * with care. - */ - @Override - public String getEncodingId() { - return delegateEncodingId(coder.getClass(), coder.getEncodingId()); - } - - /** - * {@inheritDoc} - * - * @return allowed encodings which are composed from the underlying coder class and its allowed - * encoding ids. Note that this omits any description of the coding functions. These - * should be modified with care. - */ - @Override - public Collection getAllowedEncodings() { - List allowedEncodings = Lists.newArrayList(); - for (String allowedEncoding : coder.getAllowedEncodings()) { - allowedEncodings.add(delegateEncodingId(coder.getClass(), allowedEncoding)); - } - return allowedEncodings; - } - @Override public TypeDescriptor getEncodedTypeDescriptor() { if (typeDescriptor == null) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java index 8731e5a86c056..06e7daef27121 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java @@ -87,11 +87,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index 48b7275aa3c81..cfd1979545a34 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -97,11 +97,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java index 74cd602b21e06..e2c4e28c1e523 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java @@ -24,7 +24,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -182,14 +181,4 @@ public TypeDescriptor> getEncodedTypeDescriptor() { new TypeParameter() {}, keyCoder.getEncodedTypeDescriptor()) .where(new TypeParameter() {}, valueCoder.getEncodedTypeDescriptor()); } - - @Override - public String getEncodingId() { - return ""; - } - - @Override - public Collection getAllowedEncodings() { - return Collections.emptyList(); - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index dba2a8bd396c0..1fd9a997cbfd1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -22,8 +22,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.Collection; -import java.util.Collections; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; @@ -178,14 +176,4 @@ public boolean isRegisterByteSizeObserverCheap(@Nullable T value, Context contex public TypeDescriptor getEncodedTypeDescriptor() { return valueCoder.getEncodedTypeDescriptor(); } - - @Override - public String getEncodingId() { - return ""; - } - - @Override - public Collection getAllowedEncodings() { - return Collections.emptyList(); - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java index d343af1c688d2..1a737ab3fa742 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java @@ -21,7 +21,6 @@ import java.io.InputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.io.ObjectStreamClass; import java.io.OutputStream; import java.io.Serializable; import org.apache.beam.sdk.values.TypeDescriptor; @@ -123,13 +122,6 @@ public T decode(InputStream inStream, Context context) } } - @Override - public String getEncodingId() { - return String.format("%s:%s", - type.getName(), - ObjectStreamClass.lookup(type).getSerialVersionUID()); - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java index c67fe975113fa..f8d82a55f46dd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java @@ -17,18 +17,13 @@ */ package org.apache.beam.sdk.coders; -import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.util.Structs.addList; -import static org.apache.beam.sdk.util.Structs.addString; -import static org.apache.beam.sdk.util.Structs.addStringList; -import com.google.common.collect.Lists; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import org.apache.beam.sdk.util.CloudObject; @@ -48,24 +43,11 @@ * {@link #isRegisterByteSizeObserverCheap}: the * default implementation encodes values to bytes and counts the bytes, which is considered * expensive. - *

  • {@link #getEncodingId} and {@link #getAllowedEncodings}: by default, the encoding id - * is the empty string, so only the canonical name of the subclass will be used for - * compatibility checks, and no other encoding ids are allowed.
  • * */ public abstract class StandardCoder implements Coder { protected StandardCoder() {} - @Override - public String getEncodingId() { - return ""; - } - - @Override - public Collection getAllowedEncodings() { - return Collections.emptyList(); - } - /** * Returns the list of {@link Coder Coders} that are components of this {@link Coder}. */ @@ -126,11 +108,6 @@ public String toString() { *
      *
    • component_encodings: A list of coders represented as {@link CloudObject}s * equivalent to the {@link #getCoderArguments}.
    • - *
    • encoding_id: An identifier for the binary format written by {@link #encode}. See - * {@link #getEncodingId} for further details.
    • - *
    • allowed_encodings: A collection of encodings supported by {@link #decode} in - * addition to the encoding from {@link #getEncodingId()} (which is assumed supported). - * See {@link #getAllowedEncodings} for further details.
    • *
    * *

    {@link StandardCoder} implementations should override {@link #initializeCloudObject} @@ -149,17 +126,6 @@ public final CloudObject asCloudObject() { addList(result, PropertyNames.COMPONENT_ENCODINGS, cloudComponents); } - String encodingId = getEncodingId(); - checkNotNull(encodingId, "Coder.getEncodingId() must not return null."); - if (!encodingId.isEmpty()) { - addString(result, PropertyNames.ENCODING_ID, encodingId); - } - - Collection allowedEncodings = getAllowedEncodings(); - if (!allowedEncodings.isEmpty()) { - addStringList(result, PropertyNames.ALLOWED_ENCODINGS, Lists.newArrayList(allowedEncodings)); - } - return result; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java index 51ead3c677599..39a16587cc940 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java @@ -21,7 +21,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; -import java.util.Collection; import org.apache.beam.sdk.coders.DelegateCoder.CodingFunction; import org.apache.beam.sdk.values.TypeDescriptor; @@ -121,19 +120,6 @@ public Object structuralValue(T value) { return delegateCoder.structuralValue(value); } - /** - * The encoding id is the fully qualified name of the encoded/decoded class. - */ - @Override - public String getEncodingId() { - return clazz.getName(); - } - - @Override - public Collection getAllowedEncodings() { - return delegateCoder.getAllowedEncodings(); - } - @Override public TypeDescriptor getEncodedTypeDescriptor() { return delegateCoder.getEncodedTypeDescriptor(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java index 817817b5085b4..91a46ea690932 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java @@ -64,11 +64,6 @@ public void verifyDeterministic() { StringUtf8Coder.of().verifyDeterministic(); } - @Override - public String getEncodingId() { - return ""; - } - @Override public TypeDescriptor getEncodedTypeDescriptor() { return TYPE_DESCRIPTOR; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java index 9c654a9163407..fcc0335b1fcf1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java @@ -77,11 +77,6 @@ public boolean consistentWithEquals() { return true; } - @Override - public String getEncodingId() { - return ""; - } - /** * {@inheritDoc} * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index e013c6ae3dd14..bd6d86a86ccbe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -21,7 +21,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -37,16 +36,13 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.Serializer; -import org.apache.beam.sdk.util.Structs; import org.apache.beam.sdk.util.UnownedInputStream; import org.apache.beam.sdk.util.UnownedOutputStream; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; @@ -223,32 +219,6 @@ public static void coderConsistentWithEqualsInContext( encode(coder, context, value2))); } - /** - * Verifies if a {@code Coder}'s encodingId is equal to a given - * encodingId. - */ - public static void coderHasEncodingId(Coder coder, String encodingId) throws Exception { - assertThat(coder.getEncodingId(), equalTo(encodingId)); - assertThat(Structs.getString(coder.asCloudObject(), PropertyNames.ENCODING_ID, ""), - equalTo(encodingId)); - } - - /** - * Verifies if a {@code Coder} is allowed to encode using the given - * encodingId. - */ - public static void coderAllowsEncoding(Coder coder, String encodingId) throws Exception { - assertThat(coder.getAllowedEncodings(), hasItem(encodingId)); - assertThat( - String.format("Expected to find \"%s\" in property \"%s\" of %s", - encodingId, PropertyNames.ALLOWED_ENCODINGS, coder.asCloudObject()), - Structs.getStrings( - coder.asCloudObject(), - PropertyNames.ALLOWED_ENCODINGS, - Collections.emptyList()), - hasItem(encodingId)); - } - /** * Verifies that for the given {@code Coder} and values of * type {@code T}, the structural values are equal if and only if the @@ -309,11 +279,9 @@ public static void structuralValueDecodeEncodeEqualInContext( private static final String DECODING_WIRE_FORMAT_MESSAGE = "Decoded value from known wire format does not match expected value." - + " This probably means that this Coder no longer correctly decodes" - + " a prior wire format. Changing the wire formats this Coder can read" - + " should be avoided, as it is likely to cause breakage." - + " If you truly intend to change the backwards compatibility for this Coder " - + " then you must remove any now-unsupported encodings from getAllowedEncodings()."; + + " This probably means that this Coder no longer correctly decodes" + + " a prior wire format. Changing the wire formats this Coder can read" + + " should be avoided, as it is likely to cause breakage."; public static void coderDecodesBase64(Coder coder, String base64Encoding, T value) throws Exception { @@ -334,9 +302,7 @@ public static void coderDecodesBase64( private static final String ENCODING_WIRE_FORMAT_MESSAGE = "Encoded value does not match expected wire format." + " Changing the wire format should be avoided, as it is likely to cause breakage." - + " If you truly intend to change the wire format for this Coder " - + " then you must update getEncodingId() to a new value and add any supported" - + " prior formats to getAllowedEncodings()." + + " If you truly intend to change the wire format for this Coder," + " See org.apache.beam.sdk.coders.PrintBase64Encoding for how to generate" + " new test data."; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index 34a5c5782eeb5..78a6cd13fe725 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -193,11 +193,6 @@ public boolean isRegisterByteSizeObserverCheap(long[] value, Context context) { protected long getEncodedElementByteSize(long[] value, Context context) { return VarInt.getLength(value[0]); } - - @Override - public String getEncodingId() { - return "VarLongSingletonArray"; - } }; } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java index 2a788239c97d9..cbc4d24a0ad84 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java @@ -239,12 +239,6 @@ public void testPojoEncoding() throws Exception { CoderProperties.coderDecodeEncodeEqual(coder, value); } - @Test - public void testPojoEncodingId() throws Exception { - AvroCoder coder = AvroCoder.of(Pojo.class); - CoderProperties.coderHasEncodingId(coder, Pojo.class.getName()); - } - @Test public void testGenericRecordEncoding() throws Exception { String schemaString = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java index 9db50c8f0f627..8aa260453c73d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java @@ -61,14 +61,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The definition of big endian encoding is fixed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. @@ -122,14 +114,4 @@ public void encodeNullThrowsException() throws Exception { CoderUtils.encodeToBase64(TEST_CODER, null); } - - /** - * This is a change-detector test. If this test fails, then the encoding id of - * {@link BigDecimalCoder} must change. - */ - @Test - public void testCoderIdDependencies() { - assertThat(VarIntCoder.of().getEncodingId(), equalTo("")); - assertThat(BigIntegerCoder.of().getEncodingId(), equalTo("")); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java index 1946f5a4104d7..1e98452af72c6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java @@ -51,14 +51,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The definition of big endian encoding is fixed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java index 16549679e9edb..50b40f5818d11 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java @@ -53,14 +53,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The definition of big endian is fixed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java index 6ead941a3d8d3..e2fd012e1275b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java @@ -59,14 +59,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The definition of big endian encoding is fixed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. @@ -98,14 +90,4 @@ public void encodeNullThrowsCoderException() throws Exception { CoderUtils.encodeToBase64(TEST_CODER, null); } - - /** - * This is a change-detector test. If this test fails, then the encoding id of - * {@link BigIntegerCoder} must change. - */ - @Test - public void testCoderIdDependencies() { - assertThat(VarIntCoder.of().getEncodingId(), equalTo("")); - assertThat(ByteArrayCoder.of().getEncodingId(), equalTo("")); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java index 89487d7cece63..03889ee186a6e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java @@ -107,14 +107,6 @@ private static byte[] encodeToByteArrayAndOwn( return os.toByteArray(); } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java index a4ce64c6c3f30..633ed55d61a0a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java @@ -55,14 +55,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The format is fixed by Java. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. 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 6ec258e70bd52..fe21a1c2bca50 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 @@ -30,7 +30,6 @@ import java.io.Serializable; import java.lang.reflect.Type; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -493,16 +492,6 @@ public void registerByteSizeObserver( observer.update(0L); } - @Override - public String getEncodingId() { - return getClass().getName(); - } - - @Override - public Collection getAllowedEncodings() { - return Collections.singletonList(getEncodingId()); - } - @Override public TypeDescriptor getEncodedTypeDescriptor() { return TYPE_DESCRIPTOR; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java index d05241370b19c..925454e44ecb3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java @@ -59,14 +59,6 @@ public void testDecodeEncodeContentsEqual() throws Exception { } } - // If this becomes nonempty, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java index 31bbdb9e60fb6..dfd4ea252e7f9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java @@ -85,50 +85,4 @@ public void testEncodeDecode() throws Exception { public void testEncodable() throws Exception { SerializableUtils.ensureSerializable(new MyCustomCoder("key")); } - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(new MyCustomCoder("foo"), - MyCustomCoder.class.getCanonicalName()); - } - - @Test - public void testAnonymousEncodingIdError() throws Exception { - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Anonymous CustomCoder subclass"); - thrown.expectMessage("must override getEncodingId()"); - new CustomCoder() { - - @Override - public void encode(Integer kv, OutputStream out, Context context) { - throw new UnsupportedOperationException(); - } - - @Override - public Integer decode(InputStream inStream, Context context) { - throw new UnsupportedOperationException(); - } - }.getEncodingId(); - } - - @Test - public void testAnonymousEncodingIdOk() throws Exception { - new CustomCoder() { - - @Override - public void encode(Integer kv, OutputStream out, Context context) { - throw new UnsupportedOperationException(); - } - - @Override - public Integer decode(InputStream inStream, Context context) { - throw new UnsupportedOperationException(); - } - - @Override - public String getEncodingId() { - return "A user must specify this. It can contain any character, including these: !$#%$@."; - } - }.getEncodingId(); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java index 2ae7dd571465a..5ff272fd51fc4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java @@ -28,7 +28,6 @@ import java.io.OutputStream; import java.io.Serializable; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -130,41 +129,6 @@ public void verifyDeterministic() { public List> getCoderArguments() { return Collections.emptyList(); } - - @Override - public String getEncodingId() { - return TEST_ENCODING_ID; - } - - @Override - public Collection getAllowedEncodings() { - return Collections.singletonList(TEST_ALLOWED_ENCODING); - } - } - - @Test - public void testEncodingId() throws Exception { - Coder underlyingCoder = new TestAllowedEncodingsCoder(); - - Coder trivialDelegateCoder = DelegateCoder.of( - underlyingCoder, - new DelegateCoder.CodingFunction() { - @Override - public Integer apply(Integer input) { - return input; - } - }, - new DelegateCoder.CodingFunction() { - @Override - public Integer apply(Integer input) { - return input; - } - }); - CoderProperties.coderHasEncodingId( - trivialDelegateCoder, TestAllowedEncodingsCoder.class.getName() + ":" + TEST_ENCODING_ID); - CoderProperties.coderAllowsEncoding( - trivialDelegateCoder, - TestAllowedEncodingsCoder.class.getName() + ":" + TEST_ALLOWED_ENCODING); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java index 6e21abd2b3540..679a1c74cc319 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java @@ -54,14 +54,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The format is fixed by Java. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java index 8b4bc8b398350..eb1a8984797b7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java @@ -80,14 +80,6 @@ public void testOrderedEncoding() throws Exception { Assert.assertEquals(encodings, sortedEncodings); } - // If this changes, it implies that the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java index 2eba05bdce0fe..80c3a25e0d8e3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java @@ -91,14 +91,6 @@ public void testCoderSerializable() throws Exception { CoderProperties.coderSerializable(TEST_CODER); } - // If this changes, it implies that the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java index 14c9b067777f1..4aa3d278a0345 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java @@ -97,15 +97,6 @@ public void testCoderIsSerializableWithWellKnownCoderType() throws Exception { KvCoder.of(GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE)); } - // If this changes, it implies the binary format has changed! - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId( - KvCoder.of(VarIntCoder.of(), VarIntCoder.of()), EXPECTED_ENCODING_ID); - } - /** Homogeneously typed test value for ease of use with the wire format test utility. */ private static final Coder> TEST_CODER = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/LengthPrefixCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/LengthPrefixCoderTest.java index e4564df1aec1a..27ac48a43a7a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/LengthPrefixCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/LengthPrefixCoderTest.java @@ -107,14 +107,6 @@ public void testStructuralValueConsistentWithEquals() throws Exception { } } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - @Test public void testWireFormatEncode() throws Exception { CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java index 87906e2ef2784..35239d68dd816 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java @@ -88,14 +88,6 @@ public void testCoderSerializable() throws Exception { CoderProperties.coderSerializable(TEST_CODER); } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java index 2983ea47d5173..a52e6cb9a323d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java @@ -81,14 +81,6 @@ public void testGetInstanceComponentsEmpty() { assertNull(components); } - // If this changes, it implies the binary format has changed! - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. 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 b76c0376119a9..894d2d194236e 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 @@ -73,14 +73,6 @@ public void testCoderIsSerializableWithWellKnownCoderType() throws Exception { CoderProperties.coderSerializable(NullableCoder.of(GlobalWindow.Coder.INSTANCE)); } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@code PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java index ec4b74cfa8e49..ef6df345e3fd7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java @@ -224,14 +224,6 @@ public void testMixedWithNullsEncoding() throws Exception { } } - @Test - public void testPojoEncodingId() throws Exception { - Coder coder = SerializableCoder.of(MyRecord.class); - CoderProperties.coderHasEncodingId( - coder, - String.format("%s:%s", MyRecord.class.getName(), MyRecord.serialVersionUID)); - } - @Test public void testEncodedTypeDescriptor() throws Exception { assertThat( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java index 7515553f491ea..3bf5041f71bfe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java @@ -60,14 +60,6 @@ public void testDecodeEncodeContentsEqual() throws Exception { } } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java index f2d376dc3325f..e972f07868392 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java @@ -67,12 +67,6 @@ public void testSerializable() throws Exception { CoderProperties.coderSerializable(uriCoder); } - @Test - public void testEncodingId() throws Exception { - StringDelegateCoder coder = StringDelegateCoder.of(URI.class); - CoderProperties.coderHasEncodingId(coder, URI.class.getName()); - } - @Test public void testEncodedTypeDescriptor() throws Exception { assertThat( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java index 09e1e01b02dba..1068c5f070b91 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java @@ -51,14 +51,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This should never change. The textual representation of an integer is fixed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java index be134b857a1cc..c6d961b6ad9c2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java @@ -51,14 +51,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java index f4a5280b5ad99..81e78d9193ac8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java @@ -53,14 +53,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // If this changes, it implies the binary format has changed. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index e51ee160c1275..a5f3df2dee956 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -937,11 +937,6 @@ public Accumulator decode(InputStream inStream, Coder.Context context) throws CoderException, IOException { return new Accumulator(StringUtf8Coder.of().decode(inStream, context)); } - - @Override - public String getEncodingId() { - return "CombineTest.TestKeyedCombineFn.getAccumulatorCoder()"; - } }; } } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java index a1841a10401c4..9ec7aecac193e 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java @@ -217,40 +217,6 @@ public int hashCode() { return Objects.hash(protoMessageClass, extensionHostClasses); } - /** - * The encoding identifier is designed to support evolution as per the design of Protocol - * Buffers. In order to use this class effectively, carefully follow the advice in the Protocol - * Buffers documentation at - * Updating - * A Message Type. - * - *

    In particular, the encoding identifier is guaranteed to be the same for {@link ProtoCoder} - * instances of the same principal message class, with the same registered extension host classes, - * and otherwise distinct. Note that the encoding ID does not encode any version of the message - * or extensions, nor does it include the message schema. - * - *

    When modifying a message class, here are the broadest guidelines; see the above link - * for greater detail. - * - *

      - *
    • Do not change the numeric tags for any fields. - *
    • Never remove a required field. - *
    • Only add optional or repeated fields, with sensible defaults. - *
    • When changing the type of a field, consult the Protocol Buffers documentation to ensure - * the new and old types are interchangeable. - *
    - * - *

    Code consuming this message class should be prepared to support all versions of - * the class until it is certain that no remaining serialized instances exist. - * - *

    If backwards incompatible changes must be made, the best recourse is to change the name - * of your Protocol Buffers message class. - */ - @Override - public String getEncodingId() { - return protoMessageClass.getName() + getSortedExtensionClasses().toString(); - } - @Override public void verifyDeterministic() throws NonDeterministicException { ProtobufUtil.verifyDeterministic(this); diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java index a8c6a8105442a..6d00b86633e00 100644 --- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java @@ -123,18 +123,6 @@ public void testCoderExtensionsSerialization() throws Exception { CoderProperties.coderSerializable(coder); } - @Test - public void testEncodingId() throws Exception { - Coder coderA = ProtoCoder.of(MessageA.class); - CoderProperties.coderHasEncodingId(coderA, MessageA.class.getName() + "[]"); - - ProtoCoder coder = - ProtoCoder.of(MessageC.class).withExtensionsFrom(Proto2CoderTestMessages.class); - CoderProperties.coderHasEncodingId( - coder, - String.format("%s[%s]", MessageC.class.getName(), Proto2CoderTestMessages.class.getName())); - } - @Test public void encodeNullThrowsCoderException() throws Exception { thrown.expect(CoderException.class); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java index e5f85912d4b5c..a1ca41b24431a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java @@ -86,10 +86,4 @@ public void verifyDeterministic() throws NonDeterministicException { public TypeDescriptor getEncodedTypeDescriptor() { return TYPE_DESCRIPTOR; } - - @Override - public String getEncodingId() { - return ""; - } - } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoderTest.java index f6e02dc1b5966..1fb97f5e4fdb8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoderTest.java @@ -65,14 +65,6 @@ public void testDecodeEncodeEqual() throws Exception { } } - // This identifier should only change if the JSON format of results from the BigQuery API changes. - private static final String EXPECTED_ENCODING_ID = ""; - - @Test - public void testEncodingId() throws Exception { - CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); - } - /** * Generated data to check that the wire format has not changed. To regenerate, see * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}. diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java index 1e2e07ce7efde..812bc70219caf 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java @@ -134,11 +134,6 @@ private JAXBContext getContext() throws JAXBException { return jaxbContext; } - @Override - public String getEncodingId() { - return getJAXBClass().getName(); - } - @Override public TypeDescriptor getEncodedTypeDescriptor() { return TypeDescriptor.of(jaxbClass); diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java index 276c2315d474a..940d596a0b963 100644 --- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java +++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java @@ -212,13 +212,6 @@ public void testEncodable() throws Exception { CoderProperties.coderSerializable(JAXBCoder.of(TestType.class)); } - @Test - public void testEncodingId() throws Exception { - Coder coder = JAXBCoder.of(TestType.class); - CoderProperties.coderHasEncodingId( - coder, TestType.class.getName()); - } - @Test public void testEncodedTypeDescriptor() throws Exception { assertThat(