[
https://issues.apache.org/jira/browse/BEAM-5866?focusedWorklogId=173570&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-173570
]
ASF GitHub Bot logged work on BEAM-5866:
----------------------------------------
Author: ASF GitHub Bot
Created on: 10/Dec/18 13:40
Start Date: 10/Dec/18 13:40
Worklog Time Spent: 10m
Work Description: robertwb closed pull request #6862: [BEAM-5866]
Override structuralValue in ListCoder and MapCoder
URL: https://github.com/apache/beam/pull/6862
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
index e63113819090..da90d1088bcd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.sdk.coders;
+import java.util.ArrayList;
import java.util.List;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.TypeParameter;
@@ -44,6 +45,24 @@ protected ListCoder(Coder<T> elemCoder) {
super(elemCoder, "List");
}
+ @Override
+ public boolean consistentWithEquals() {
+ return getElemCoder().consistentWithEquals();
+ }
+
+ @Override
+ public Object structuralValue(List<T> values) {
+ if (consistentWithEquals()) {
+ return values;
+ } else {
+ List<Object> ret = new ArrayList<>(values.size());
+ for (T value : values) {
+ ret.add(getElemCoder().structuralValue(value));
+ }
+ return ret;
+ }
+ }
+
/**
* List sizes are always known, so ListIterable may be deterministic while
the general
* IterableLikeCoder is not.
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 24052ed501a1..4ca3fcc8e05b 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
@@ -147,6 +147,25 @@ public void verifyDeterministic() throws
NonDeterministicException {
this, "Ordering of entries in a Map may be non-deterministic.");
}
+ @Override
+ public boolean consistentWithEquals() {
+ return keyCoder.consistentWithEquals() &&
valueCoder.consistentWithEquals();
+ }
+
+ @Override
+ public Object structuralValue(Map<K, V> value) {
+ if (consistentWithEquals()) {
+ return value;
+ } else {
+ Map<Object, Object> ret = Maps.newHashMapWithExpectedSize(value.size());
+ for (Map.Entry<K, V> entry : value.entrySet()) {
+ ret.put(
+ keyCoder.structuralValue(entry.getKey()),
valueCoder.structuralValue(entry.getValue()));
+ }
+ return ret;
+ }
+ }
+
@Override
public void registerByteSizeObserver(Map<K, V> map, ElementByteSizeObserver
observer)
throws Exception {
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 334ff6bac7db..e793e40935e8 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
@@ -17,7 +17,9 @@
*/
package org.apache.beam.sdk.coders;
+import static junit.framework.TestCase.assertTrue;
import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import java.util.ArrayList;
@@ -110,6 +112,25 @@ public void testListWithNullsAndSerializableCoder() throws
Exception {
CoderProperties.coderDecodeEncodeEqual(coder, list);
}
+ @Test
+ public void testStructuralValueDecodeEncodeEqual() throws Exception {
+ ListCoder<byte[]> coder = ListCoder.of(ByteArrayCoder.of());
+ List<byte[]> value = Collections.singletonList(new byte[] {1, 2, 3, 4});
+ CoderProperties.structuralValueDecodeEncodeEqual(coder, value);
+ }
+
+ @Test
+ public void testNotConsistentWithEquals() {
+ ListCoder<byte[]> coder = ListCoder.of(ByteArrayCoder.of());
+ assertFalse(coder.consistentWithEquals());
+ }
+
+ @Test
+ public void testConsistentWithEquals() {
+ ListCoder<Integer> coder = ListCoder.of(VarIntCoder.of());
+ assertTrue(coder.consistentWithEquals());
+ }
+
@Test
public void testEncodedTypeDescriptor() throws Exception {
TypeDescriptor<List<Integer>> typeDescriptor = new
TypeDescriptor<List<Integer>>() {};
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 578b4b773710..f18f2b220b8a 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
@@ -18,7 +18,9 @@
package org.apache.beam.sdk.coders;
import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
import com.google.common.collect.ImmutableMap;
import java.util.Arrays;
@@ -84,6 +86,25 @@ public void encodeNullThrowsCoderException() throws
Exception {
CoderUtils.encodeToBase64(TEST_CODER, null);
}
+ @Test
+ public void testStructuralValueDecodeEncodeEqual() throws Exception {
+ MapCoder<byte[], Integer> coder = MapCoder.of(ByteArrayCoder.of(),
VarIntCoder.of());
+ Map<byte[], Integer> value = Collections.singletonMap(new byte[] {1, 2, 3,
4}, 1);
+ CoderProperties.structuralValueDecodeEncodeEqual(coder, value);
+ }
+
+ @Test
+ public void testNotConsistentWithEquals() {
+ MapCoder<Integer, byte[]> coder = MapCoder.of(VarIntCoder.of(),
ByteArrayCoder.of());
+ assertFalse(coder.consistentWithEquals());
+ }
+
+ @Test
+ public void testConsistentWithEquals() {
+ MapCoder<Integer, Integer> coder = MapCoder.of(VarIntCoder.of(),
VarIntCoder.of());
+ assertTrue(coder.consistentWithEquals());
+ }
+
@Test
public void testEncodedTypeDescriptor() throws Exception {
TypeDescriptor<Map<Integer, String>> typeDescriptor =
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 173570)
Time Spent: 7.5h (was: 7h 20m)
> RowCoder doesn't implement structuralValue
> ------------------------------------------
>
> Key: BEAM-5866
> URL: https://issues.apache.org/jira/browse/BEAM-5866
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-core
> Reporter: Gleb Kanterov
> Assignee: Gleb Kanterov
> Priority: Major
> Time Spent: 7.5h
> Remaining Estimate: 0h
>
> These two properties fail for RowCoder with `BYTES` field, or `Map<BYTES, ?>`
> field.
> {code}
> public static <T> void testConsistentWithEquals(Coder<T> coder, T example) {
> assumeTrue(coder.consistentWithEquals());
> byte[] bytes = encodeBytes(coder, example);
> // even if the coder is non-deterministic, if the encoded bytes match,
> // coder is consistent with equals, decoded values must be equal
> T out0 = decodeBytes(coder, bytes);
> T out1 = decodeBytes(coder, bytes);
> assertEquals("If the encoded bytes match, decoded values must be equal",
> out0, out1);
> assertEquals(
> "If two values are equal, their hash codes must be equal",
> out0.hashCode(),
> out1.hashCode());
> }
> public static <T> void testStructuralValueConsistentWithEquals(Coder<T>
> coder, T example) {
> byte[] bytes = encodeBytes(coder, example);
> // even if coder is non-deterministic, if the encoded bytes match,
> // structural values must be equal
> Object out0 = coder.structuralValue(decodeBytes(coder, bytes));
> Object out1 = coder.structuralValue(decodeBytes(coder, bytes));
> assertEquals("If the encoded bytes match, structural values must be
> equal", out0, out1);
> assertEquals(
> "If two values are equal, their hash codes must be equal",
> out0.hashCode(),
> out1.hashCode());
> }
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)