Skip to content

Commit

Permalink
Fix Reshuffle implementation in Java SDK
Browse files Browse the repository at this point in the history
  • Loading branch information
kennknowles committed Oct 5, 2023
1 parent c89d44d commit c19a102
Showing 1 changed file with 92 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,24 @@
*/
package org.apache.beam.sdk.transforms;

import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger;
import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
import org.apache.beam.sdk.util.IdentityWindowFn;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TimestampedValue;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedInteger;
import org.checkerframework.checker.nullness.qual.Nullable;
Expand Down Expand Up @@ -81,28 +89,103 @@ public PCollection<KV<K, V>> expand(PCollection<KV<K, V>> input) {
.withTimestampCombiner(TimestampCombiner.EARLIEST)
.withAllowedLateness(Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()));

return input
.apply(rewindow)
.apply("ReifyOriginalTimestamps", Reify.timestampsInValue())
.apply(GroupByKey.create())
PCollection<KV<K, ValueInSingleWindow<V>>> reified =
input.apply(rewindow).apply("ReifyOriginalMetadata", Reify.windowsInValue());

PCollection<KV<K, Iterable<ValueInSingleWindow<V>>>> grouped =
reified.apply(GroupByKey.create());
return grouped
// Set the windowing strategy directly, so that it doesn't get counted as the user having
// set allowed lateness.
.setWindowingStrategyInternal(originalStrategy)
.apply(
"ExpandIterable",
ParDo.of(
new DoFn<KV<K, Iterable<TimestampedValue<V>>>, KV<K, TimestampedValue<V>>>() {
new DoFn<KV<K, Iterable<ValueInSingleWindow<V>>>, KV<K, ValueInSingleWindow<V>>>() {
@ProcessElement
public void processElement(
@Element KV<K, Iterable<TimestampedValue<V>>> element,
OutputReceiver<KV<K, TimestampedValue<V>>> r) {
@Element KV<K, Iterable<ValueInSingleWindow<V>>> element,
OutputReceiver<KV<K, ValueInSingleWindow<V>>> r) {
K key = element.getKey();
for (TimestampedValue<V> value : element.getValue()) {
for (ValueInSingleWindow<V> value : element.getValue()) {
r.output(KV.of(key, value));
}
}
}))
.apply("RestoreOriginalTimestamps", ReifyTimestamps.extractFromValues());
.apply(Window.into(new RestoreWindowsFn<>(originalStrategy.getWindowFn())))
.apply("RestoreOriginalTimestamps", new RestoreTimestamps<>());
}

private static class RestoreWindowsFn<K, V, W extends BoundedWindow>
extends NonMergingWindowFn<KV<K, ValueInSingleWindow<V>>, W> {

private final WindowFn<?, W> originalWindowFn;

private RestoreWindowsFn(WindowFn<?, W> originalWindowFn) {
this.originalWindowFn = originalWindowFn;
}

@Override
public Collection<W> assignWindows(AssignContext c) throws Exception {
return Collections.singleton((W) c.element().getValue().getWindow());
}

@Override
public boolean isCompatible(WindowFn<?, ?> other) {
throw new UnsupportedOperationException(
String.format(
"%s.isCompatible() should never be called."
+ " It is a private implementation detail of sdk utilities."
+ " This message indicates a bug in the Beam SDK.",
getClass().getCanonicalName()));
}

@Override
public void verifyCompatibility(WindowFn<?, ?> other) throws IncompatibleWindowException {
throw new UnsupportedOperationException(
String.format(
"%s.verifyCompatibility() should never be called."
+ " It is a private implementation detail of sdk utilities."
+ " This message indicates a bug in the Beam SDK.",
getClass().getCanonicalName()));
}

@Override
public Coder<W> windowCoder() {
return originalWindowFn.windowCoder();
}

@Override
public WindowMappingFn<W> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException(
String.format(
"%s.getSideInputWindow() should never be called."
+ " It is a private implementation detail of sdk utilities."
+ " This message indicates a bug in the Beam SDK.",
getClass().getCanonicalName()));
}
}

private static class RestoreTimestamps<K, V>
extends PTransform<PCollection<KV<K, ValueInSingleWindow<V>>>, PCollection<KV<K, V>>> {
@Override
public PCollection<KV<K, V>> expand(PCollection<KV<K, ValueInSingleWindow<V>>> input) {
return input.apply(
ParDo.of(
new DoFn<KV<K, ValueInSingleWindow<V>>, KV<K, V>>() {
@Override
public Duration getAllowedTimestampSkew() {
return Duration.millis(Long.MAX_VALUE);
}

@ProcessElement
public void processElement(
@Element KV<K, TimestampedValue<V>> kv, OutputReceiver<KV<K, V>> r) {
r.outputWithTimestamp(
KV.of(kv.getKey(), kv.getValue().getValue()), kv.getValue().getTimestamp());
}
}));
}
}

/** Implementation of {@link #viaRandomKey()}. */
Expand Down

0 comments on commit c19a102

Please sign in to comment.