[ https://issues.apache.org/jira/browse/BEAM-5184?focusedWorklogId=137864&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-137864 ]
ASF GitHub Bot logged work on BEAM-5184: ---------------------------------------- Author: ASF GitHub Bot Created on: 24/Aug/18 15:57 Start Date: 24/Aug/18 15:57 Worklog Time Spent: 10m Work Description: lukecwik closed pull request #6257: [BEAM-5184] Multimap side inputs with duplicate keys and values are being lost URL: https://github.com/apache/beam/pull/6257 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/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 145a6a23e1b..74ffac74378 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -22,8 +22,8 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ForwardingMap; -import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -637,7 +637,7 @@ public void processElement(ProcessContext c) throws Exception { public void processElement(ProcessContext c) throws Exception { Optional<Object> previousWindowStructuralValue = Optional.absent(); Optional<W> previousWindow = Optional.absent(); - Multimap<K, WindowedValue<V>> multimap = HashMultimap.create(); + Multimap<K, WindowedValue<V>> multimap = ArrayListMultimap.create(); for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) { Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey()); if (previousWindowStructuralValue.isPresent() @@ -652,7 +652,7 @@ public void processElement(ProcessContext c) throws Exception { valueInEmptyWindows( new TransformedMap<>( IterableWithWindowedValuesToIterable.of(), resultMap)))); - multimap = HashMultimap.create(); + multimap = ArrayListMultimap.create(); } multimap.put( diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java index b401d6d4e6d..2418825da20 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java @@ -568,6 +568,12 @@ public void testToMultimapDoFn() throws Exception { 1, (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of( + KV.of( + windowA, + WindowedValue.of( + KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), + // [BEAM-5184] Specifically test with a duplicate value to ensure that + // duplicate key/values are not lost. KV.of( windowA, WindowedValue.of( @@ -613,7 +619,7 @@ public void testToMultimapDoFn() throws Exception { outputMap = output.get(0).getValue().getValue(); assertEquals(2, outputMap.size()); - assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L)); + assertThat(outputMap.get(1L), containsInAnyOrder(11L, 11L, 12L)); assertThat(outputMap.get(2L), containsInAnyOrder(21L)); outputMap = output.get(1).getValue().getValue(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java index b275e03c57f..906ddecf72b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.values; import com.google.common.base.MoreObjects; -import com.google.common.collect.HashMultimap; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; @@ -290,7 +290,7 @@ public int hashCode() { public Map<K, Iterable<V>> apply(MultimapView<Void, KV<K, V>> primitiveViewT) { // TODO: BEAM-3071 - fix this so that we aren't relying on Java equality and are // using structural value equality. - Multimap<K, V> multimap = HashMultimap.create(); + Multimap<K, V> multimap = ArrayListMultimap.create(); for (KV<K, V> elem : primitiveViewT.get(null)) { multimap.put(elem.getKey(), elem.getValue()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 84e26878b03..dc3bec854c7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -508,7 +508,9 @@ public void testMultimapSideInput() { final PCollectionView<Map<String, Iterable<Integer>>> view = pipeline - .apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) + .apply( + "CreateSideInput", + Create.of(KV.of("a", 1), KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) .apply(View.asMultimap()); PCollection<KV<String, Integer>> output = @@ -529,7 +531,11 @@ public void processElement(ProcessContext c) { PAssert.that(output) .containsInAnyOrder( - KV.of("apple", 1), KV.of("apple", 2), KV.of("banana", 3), KV.of("blackberry", 3)); + KV.of("apple", 1), + KV.of("apple", 1), + KV.of("apple", 2), + KV.of("banana", 3), + KV.of("blackberry", 3)); pipeline.run(); } @@ -540,7 +546,9 @@ public void testMultimapAsEntrySetSideInput() { final PCollectionView<Map<String, Iterable<Integer>>> view = pipeline - .apply("CreateSideInput", Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) + .apply( + "CreateSideInput", + Create.of(KV.of("a", 1), KV.of("a", 1), KV.of("a", 2), KV.of("b", 3))) .apply(View.asMultimap()); PCollection<KV<String, Integer>> output = @@ -564,7 +572,8 @@ public void processElement(ProcessContext c) { }) .withSideInputs(view)); - PAssert.that(output).containsInAnyOrder(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)); + PAssert.that(output) + .containsInAnyOrder(KV.of("a", 1), KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)); pipeline.run(); } @@ -607,7 +616,7 @@ public void testMultimapSideInputWithNonDeterministicKeyCoder() { pipeline .apply( "CreateSideInput", - Create.of(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)) + Create.of(KV.of("a", 1), KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)) .withCoder(KvCoder.of(new NonDeterministicStringCoder(), VarIntCoder.of()))) .apply(View.asMultimap()); @@ -629,7 +638,11 @@ public void processElement(ProcessContext c) { PAssert.that(output) .containsInAnyOrder( - KV.of("apple", 1), KV.of("apple", 2), KV.of("banana", 3), KV.of("blackberry", 3)); + KV.of("apple", 1), + KV.of("apple", 1), + KV.of("apple", 2), + KV.of("banana", 3), + KV.of("blackberry", 3)); pipeline.run(); } @@ -644,6 +657,7 @@ public void testWindowedMultimapSideInput() { "CreateSideInput", Create.timestamped( TimestampedValue.of(KV.of("a", 1), new Instant(1)), + TimestampedValue.of(KV.of("a", 1), new Instant(2)), TimestampedValue.of(KV.of("a", 2), new Instant(7)), TimestampedValue.of(KV.of("b", 3), new Instant(14)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) @@ -673,7 +687,11 @@ public void processElement(ProcessContext c) { PAssert.that(output) .containsInAnyOrder( - KV.of("apple", 1), KV.of("apple", 2), KV.of("banana", 3), KV.of("blackberry", 3)); + KV.of("apple", 1), + KV.of("apple", 1), + KV.of("apple", 2), + KV.of("banana", 3), + KV.of("blackberry", 3)); pipeline.run(); } @@ -688,6 +706,7 @@ public void testWindowedMultimapAsEntrySetSideInput() { "CreateSideInput", Create.timestamped( TimestampedValue.of(KV.of("a", 1), new Instant(1)), + TimestampedValue.of(KV.of("a", 1), new Instant(2)), TimestampedValue.of(KV.of("a", 2), new Instant(7)), TimestampedValue.of(KV.of("b", 3), new Instant(14)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) @@ -719,7 +738,8 @@ public void processElement(ProcessContext c) { }) .withSideInputs(view)); - PAssert.that(output).containsInAnyOrder(KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)); + PAssert.that(output) + .containsInAnyOrder(KV.of("a", 1), KV.of("a", 1), KV.of("a", 2), KV.of("b", 3)); pipeline.run(); } @@ -734,6 +754,7 @@ public void testWindowedMultimapSideInputWithNonDeterministicKeyCoder() { "CreateSideInput", Create.timestamped( TimestampedValue.of(KV.of("a", 1), new Instant(1)), + TimestampedValue.of(KV.of("a", 1), new Instant(2)), TimestampedValue.of(KV.of("a", 2), new Instant(7)), TimestampedValue.of(KV.of("b", 3), new Instant(14))) .withCoder(KvCoder.of(new NonDeterministicStringCoder(), VarIntCoder.of()))) @@ -764,7 +785,11 @@ public void processElement(ProcessContext c) { PAssert.that(output) .containsInAnyOrder( - KV.of("apple", 1), KV.of("apple", 2), KV.of("banana", 3), KV.of("blackberry", 3)); + KV.of("apple", 1), + KV.of("apple", 1), + KV.of("apple", 2), + KV.of("banana", 3), + KV.of("blackberry", 3)); pipeline.run(); } ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 137864) Time Spent: 5h 10m (was: 5h) > Multimap side inputs with duplicate keys and values are being lost > ------------------------------------------------------------------ > > Key: BEAM-5184 > URL: https://issues.apache.org/jira/browse/BEAM-5184 > Project: Beam > Issue Type: Bug > Components: sdk-java-core > Reporter: Luke Cwik > Assignee: Vaclav Plajt > Priority: Major > Time Spent: 5h 10m > Remaining Estimate: 0h > > Side inputs with duplicate values are being lost due to the usage of a set > based multimap. > [https://github.com/apache/beam/blob/05fb694f265dda0254d7256e938e508fec9ba098/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java#L293] > > Originating thread: > [https://lists.apache.org/thread.html/48bae7cf71bf6851622cdee0e8bc8619c79c4c2273ed63f288202169@%3Cdev.beam.apache.org%3E] > > Please update the existing tests to exercise this scenario as well: > https://github.com/apache/beam/blob/9f23ffc97535e7255245f3852b9d2f0939df5a0a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java#L507 -- This message was sent by Atlassian JIRA (v7.6.3#76005)