Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-5184] Multimap side inputs with duplicate keys and values are being lost #6257

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -637,7 +637,7 @@ static class ToMultimapDoFn<K, V, W extends BoundedWindow>
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()
Expand All @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -290,7 +290,7 @@ public Materialization<MultimapView<Void, KV<K, V>>> getMaterialization() {
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());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 =
Expand All @@ -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();
}
Expand All @@ -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 =
Expand All @@ -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();
}
Expand Down Expand Up @@ -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());

Expand All @@ -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();
}
Expand All @@ -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))))
Expand Down Expand Up @@ -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();
}
Expand All @@ -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))))
Expand Down Expand Up @@ -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();
}
Expand All @@ -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())))
Expand Down Expand Up @@ -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();
}
Expand Down