Skip to content

Commit 43db80d

Browse files
AlexKbitkennknowles
authored andcommitted
[BEAM-3279] Deprecate and remove Coder.consistentWithEquals (apache#8071)
1 parent c8bceba commit 43db80d

39 files changed

Lines changed: 316 additions & 184 deletions

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -79,16 +79,6 @@ public void verifyDeterministic() throws NonDeterministicException {
7979
BIG_INT_CODER.verifyDeterministic();
8080
}
8181

82-
/**
83-
* {@inheritDoc}
84-
*
85-
* @return {@code true}. This coder is injective.
86-
*/
87-
@Override
88-
public boolean consistentWithEquals() {
89-
return true;
90-
}
91-
9282
/**
9383
* {@inheritDoc}
9484
*
@@ -112,4 +102,10 @@ protected long getEncodedElementByteSize(BigDecimal value) throws Exception {
112102
return VAR_INT_CODER.getEncodedElementByteSize(value.scale())
113103
+ BIG_INT_CODER.getEncodedElementByteSize(value.unscaledValue());
114104
}
105+
106+
/** {@inheritDoc} */
107+
@Override
108+
public Object structuralValue(BigDecimal value) {
109+
return value;
110+
}
115111
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -62,16 +62,6 @@ public Integer decode(InputStream inStream) throws IOException, CoderException {
6262
@Override
6363
public void verifyDeterministic() {}
6464

65-
/**
66-
* {@inheritDoc}
67-
*
68-
* @return {@code true}. This coder is injective.
69-
*/
70-
@Override
71-
public boolean consistentWithEquals() {
72-
return true;
73-
}
74-
7565
/**
7666
* {@inheritDoc}
7767
*
@@ -99,4 +89,10 @@ protected long getEncodedElementByteSize(Integer value) throws Exception {
9989
}
10090
return 4;
10191
}
92+
93+
/** {@inheritDoc} */
94+
@Override
95+
public Object structuralValue(Integer value) {
96+
return value;
97+
}
10298
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -62,16 +62,6 @@ public Long decode(InputStream inStream) throws IOException, CoderException {
6262
@Override
6363
public void verifyDeterministic() {}
6464

65-
/**
66-
* {@inheritDoc}
67-
*
68-
* @return {@code true}. This coder is injective.
69-
*/
70-
@Override
71-
public boolean consistentWithEquals() {
72-
return true;
73-
}
74-
7565
/**
7666
* {@inheritDoc}
7767
*
@@ -99,4 +89,10 @@ protected long getEncodedElementByteSize(Long value) throws Exception {
9989
}
10090
return 8;
10191
}
92+
93+
/** {@inheritDoc} */
94+
@Override
95+
public Object structuralValue(Long value) {
96+
return value;
97+
}
10298
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianShortCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -62,16 +62,6 @@ public Short decode(InputStream inStream) throws IOException, CoderException {
6262
@Override
6363
public void verifyDeterministic() {}
6464

65-
/**
66-
* {@inheritDoc}
67-
*
68-
* @return {@code true}. This coder is injective.
69-
*/
70-
@Override
71-
public boolean consistentWithEquals() {
72-
return true;
73-
}
74-
7565
/**
7666
* {@inheritDoc}
7767
*
@@ -99,4 +89,10 @@ protected long getEncodedElementByteSize(Short value) throws Exception {
9989
}
10090
return 2;
10191
}
92+
93+
/** {@inheritDoc} */
94+
@Override
95+
public Object structuralValue(Short value) {
96+
return value;
97+
}
10298
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -69,16 +69,6 @@ public void verifyDeterministic() {
6969
BYTE_ARRAY_CODER.verifyDeterministic();
7070
}
7171

72-
/**
73-
* {@inheritDoc}
74-
*
75-
* @return {@code true}. This coder is injective.
76-
*/
77-
@Override
78-
public boolean consistentWithEquals() {
79-
return true;
80-
}
81-
8272
/**
8373
* {@inheritDoc}
8474
*
@@ -99,4 +89,10 @@ protected long getEncodedElementByteSize(BigInteger value) throws Exception {
9989
checkNotNull(value, String.format("cannot encode a null %s", BigInteger.class.getSimpleName()));
10090
return BYTE_ARRAY_CODER.getEncodedElementByteSize(value.toByteArray());
10191
}
92+
93+
/** {@inheritDoc} */
94+
@Override
95+
public Object structuralValue(BigInteger value) {
96+
return value;
97+
}
10298
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitSetCoder.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,9 @@ public void verifyDeterministic() throws NonDeterministicException {
6363
this, "BitSetCoder requires its ByteArrayCoder to be deterministic.", BYTE_ARRAY_CODER);
6464
}
6565

66+
/** {@inheritDoc} */
6667
@Override
67-
public boolean consistentWithEquals() {
68-
return true;
68+
public Object structuralValue(BitSet value) {
69+
return value;
6970
}
7071
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BooleanCoder.java

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -42,11 +42,6 @@ public Boolean decode(InputStream is) throws IOException {
4242
return BYTE_CODER.decode(is) == 1;
4343
}
4444

45-
@Override
46-
public boolean consistentWithEquals() {
47-
return true;
48-
}
49-
5045
@Override
5146
public boolean isRegisterByteSizeObserverCheap(Boolean value) {
5247
return true;
@@ -56,4 +51,10 @@ public boolean isRegisterByteSizeObserverCheap(Boolean value) {
5651
protected long getEncodedElementByteSize(Boolean value) throws Exception {
5752
return 1;
5853
}
54+
55+
/** {@inheritDoc} */
56+
@Override
57+
public Object structuralValue(Boolean value) {
58+
return value;
59+
}
5960
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -71,16 +71,6 @@ public Byte decode(InputStream inStream) throws IOException, CoderException {
7171
@Override
7272
public void verifyDeterministic() {}
7373

74-
/**
75-
* {@inheritDoc}
76-
*
77-
* @return {@code true}. This coder is injective.
78-
*/
79-
@Override
80-
public boolean consistentWithEquals() {
81-
return true;
82-
}
83-
8474
/**
8575
* {@inheritDoc}
8676
*
@@ -108,4 +98,10 @@ protected long getEncodedElementByteSize(Byte value) throws Exception {
10898
}
10999
return 1;
110100
}
101+
102+
/** {@inheritDoc} */
103+
@Override
104+
public Object structuralValue(Byte value) {
105+
return value;
106+
}
111107
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -210,15 +210,20 @@ public static void verifyDeterministic(Coder<?> target, String message, Coder<?>
210210
/**
211211
* Returns {@code true} if this {@link Coder} is injective with respect to {@link Objects#equals}.
212212
*
213+
* <p>By default, returns false, assuming that the encoding may differ from {@link Objects#equals}
214+
* in arbitrary ways.
215+
*
213216
* <p>Whenever the encoded bytes of two values are equal, then the original values are equal
214217
* according to {@code Objects.equals()}. Note that this is well-defined for {@code null}.
215218
*
216-
* <p>This condition is most notably false for arrays. More generally, this condition is false
217-
* whenever {@code equals()} compares object identity, rather than performing a
218-
* semantic/structural comparison.
219+
* <p>This condition is most notably false for arrays, where encoded equality does not correspond
220+
* to Java equality. More generally, this condition is false whenever {@code equals()} compares
221+
* object identity, rather than performing a semantic/structural comparison.
219222
*
220-
* <p>By default, returns false.
223+
* @deprecated For coders that are injective with respect to {@link Objects#equals}, override
224+
* {@link #structuralValue(Object)} to return the input element directly.
221225
*/
226+
@Deprecated
222227
public boolean consistentWithEquals() {
223228
return false;
224229
}

sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -72,16 +72,6 @@ public void verifyDeterministic() throws NonDeterministicException {
7272
this, "Floating point encodings are not guaranteed to be deterministic.");
7373
}
7474

75-
/**
76-
* {@inheritDoc}
77-
*
78-
* @return {@code true}. This coder is injective.
79-
*/
80-
@Override
81-
public boolean consistentWithEquals() {
82-
return true;
83-
}
84-
8575
/**
8676
* {@inheritDoc}
8777
*
@@ -109,4 +99,10 @@ protected long getEncodedElementByteSize(Double value) throws Exception {
10999
}
110100
return 8;
111101
}
102+
103+
/** {@inheritDoc} */
104+
@Override
105+
public Object structuralValue(Double value) {
106+
return value;
107+
}
112108
}

0 commit comments

Comments
 (0)