Skip to content
This repository was archived by the owner on Jul 1, 2025. It is now read-only.

Commit 3d46caf

Browse files
authored
[NEMO-460] Setting coders in CombinePerKey transformation (#303)
JIRA: [NEMO-460: Setting coders in CombinePerKey transformation](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-460) **Major changes:** - Added the additional parameter "inputCoder" for GBKTransform constructor. - Fixed the input coder and the output coder for the partial combine transform and the final combine transform. **Minor changes to note:** - Fixed the main output TupleTags for the partial combine transform and the final combine transform. **Tests for the changes:** - Current tests suffice. **Other comments:** - This needs to be merged after merging #302 Closes #303
1 parent e325087 commit 3d46caf

File tree

3 files changed

+23
-18
lines changed

3 files changed

+23
-18
lines changed

compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -406,10 +406,11 @@ private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslato
406406
KvCoder.of(inputCoder.getKeyCoder(),
407407
accumulatorCoder),
408408
null, mainInput.getWindowingStrategy()));
409+
final TupleTag<?> partialMainOutputTag = new TupleTag<>();
409410
final GBKTransform partialCombineStreamTransform =
410-
new GBKTransform(
411-
getOutputCoders(pTransform),
412-
new TupleTag<>(),
411+
new GBKTransform(inputCoder,
412+
Collections.singletonMap(partialMainOutputTag, KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)),
413+
partialMainOutputTag,
413414
mainInput.getWindowingStrategy(),
414415
ctx.getPipelineOptions(),
415416
partialSystemReduceFn,
@@ -418,9 +419,9 @@ private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslato
418419
true);
419420

420421
final GBKTransform finalCombineStreamTransform =
421-
new GBKTransform(
422+
new GBKTransform(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
422423
getOutputCoders(pTransform),
423-
new TupleTag<>(),
424+
Iterables.getOnlyElement(beamNode.getOutputs().keySet()),
424425
mainInput.getWindowingStrategy(),
425426
ctx.getPipelineOptions(),
426427
finalSystemReduceFn,
@@ -556,14 +557,15 @@ private static Transform createGBKTransform(
556557
final AppliedPTransform<?, ?, ?> pTransform = beamNode.toAppliedPTransform(ctx.getPipeline());
557558
final PCollection<?> mainInput = (PCollection<?>)
558559
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));
559-
final TupleTag mainOutputTag = new TupleTag<>();
560+
final TupleTag mainOutputTag = Iterables.getOnlyElement(beamNode.getOutputs().keySet());
560561

561562
if (isGlobalWindow(beamNode, ctx.getPipeline())) {
562563
// GroupByKey Transform when using a global windowing strategy.
563564
return new GroupByKeyTransform();
564565
} else {
565566
// GroupByKey Transform when using a non-global windowing strategy.
566567
return new GBKTransform<>(
568+
(KvCoder) mainInput.getCoder(),
567569
getOutputCoders(pTransform),
568570
mainOutputTag,
569571
mainInput.getWindowingStrategy(),

compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/transform/GBKTransform.java

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,8 @@ public final class GBKTransform<K, InputT, OutputT>
5858
private transient OutputCollector originOc;
5959
private final boolean isPartialCombining;
6060

61-
public GBKTransform(final Map<TupleTag<?>, Coder<?>> outputCoders,
61+
public GBKTransform(final Coder<KV<K, InputT>> inputCoder,
62+
final Map<TupleTag<?>, Coder<?>> outputCoders,
6263
final TupleTag<KV<K, OutputT>> mainOutputTag,
6364
final WindowingStrategy<?, ?> windowingStrategy,
6465
final PipelineOptions options,
@@ -67,7 +68,7 @@ public GBKTransform(final Map<TupleTag<?>, Coder<?>> outputCoders,
6768
final DisplayData displayData,
6869
final boolean isPartialCombining) {
6970
super(null,
70-
null,
71+
inputCoder,
7172
outputCoders,
7273
mainOutputTag,
7374
Collections.emptyList(), /* no additional outputs */
@@ -278,7 +279,7 @@ public GBKOutputCollector(final OutputCollector oc) {
278279

279280
/** Emit output. If {@param output} is emitted on-time, save its timestamp in the output watermark map. */
280281
@Override
281-
public void emit(final WindowedValue<KV<K, OutputT>> output) {
282+
public final void emit(final WindowedValue<KV<K, OutputT>> output) {
282283
// The watermark advances only in ON_TIME
283284
if (output.getPane().getTiming().equals(PaneInfo.Timing.ON_TIME)) {
284285
KV<K, OutputT> value = output.getValue();
@@ -296,13 +297,13 @@ public void emit(final WindowedValue<KV<K, OutputT>> output) {
296297

297298
/** Emit watermark. */
298299
@Override
299-
public void emitWatermark(final Watermark watermark) {
300+
public final void emitWatermark(final Watermark watermark) {
300301
oc.emitWatermark(watermark);
301302
}
302303

303304
/** Emit output value to {@param dstVertexId}. */
304305
@Override
305-
public <T> void emit(final String dstVertexId, final T output) {
306+
public final <T> void emit(final String dstVertexId, final T output) {
306307
oc.emit(dstVertexId, output);
307308
}
308309
}

compiler/test/src/test/java/org/apache/nemo/compiler/frontend/beam/transform/GBKTransformTest.java

Lines changed: 9 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
*/
1919
package org.apache.nemo.compiler.frontend.beam.transform;
2020

21+
import com.google.common.collect.Iterables;
2122
import junit.framework.TestCase;
2223
import org.apache.beam.runners.core.SystemReduceFn;
2324
import org.apache.beam.sdk.coders.*;
@@ -41,15 +42,12 @@
4142

4243
import static org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing.*;
4344
import static org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES;
44-
import static org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode.DISCARDING_FIRED_PANES;
45-
import static org.junit.Assert.assertEquals;
4645
import static org.mockito.Mockito.mock;
4746

4847
public class GBKTransformTest extends TestCase {
4948
private static final Logger LOG = LoggerFactory.getLogger(GBKTransformTest.class.getName());
5049
private final static Coder STRING_CODER = StringUtf8Coder.of();
5150
private final static Coder INTEGER_CODER = BigEndianIntegerCoder.of();
52-
private final static Map<TupleTag<?>, Coder<?>> NULL_OUTPUT_CODERS = null;
5351

5452
private void checkOutput(final KV<String, Integer> expected, final KV<String, Integer> result) {
5553
// check key
@@ -155,7 +153,8 @@ public void test_combine() {
155153

156154
final GBKTransform<String, Integer, Integer> combine_transform =
157155
new GBKTransform(
158-
NULL_OUTPUT_CODERS,
156+
KvCoder.of(STRING_CODER, INTEGER_CODER),
157+
Collections.singletonMap(outputTag, KvCoder.of(STRING_CODER, INTEGER_CODER)),
159158
outputTag,
160159
WindowingStrategy.of(slidingWindows).withMode(ACCUMULATING_FIRED_PANES),
161160
PipelineOptionsFactory.as(NemoPipelineOptions.class),
@@ -283,7 +282,8 @@ public void test_combine_lateData() {
283282

284283
final GBKTransform<String, Integer, Integer> combine_transform =
285284
new GBKTransform(
286-
NULL_OUTPUT_CODERS,
285+
KvCoder.of(STRING_CODER, INTEGER_CODER),
286+
Collections.singletonMap(outputTag, KvCoder.of(STRING_CODER, INTEGER_CODER)),
287287
outputTag,
288288
WindowingStrategy.of(slidingWindows).withMode(ACCUMULATING_FIRED_PANES).withAllowedLateness(lateness),
289289
PipelineOptionsFactory.as(NemoPipelineOptions.class),
@@ -377,7 +377,8 @@ public void test_gbk() {
377377

378378
final GBKTransform<String, String, Iterable<String>> doFnTransform =
379379
new GBKTransform(
380-
NULL_OUTPUT_CODERS,
380+
KvCoder.of(STRING_CODER, STRING_CODER),
381+
Collections.singletonMap(outputTag, KvCoder.of(STRING_CODER, IterableCoder.of(STRING_CODER))),
381382
outputTag,
382383
WindowingStrategy.of(slidingWindows),
383384
PipelineOptionsFactory.as(NemoPipelineOptions.class),
@@ -562,7 +563,8 @@ public void test_gbk_eventTimeTrigger() {
562563

563564
final GBKTransform<String, String, Iterable<String>> doFnTransform =
564565
new GBKTransform(
565-
NULL_OUTPUT_CODERS,
566+
KvCoder.of(STRING_CODER, STRING_CODER),
567+
Collections.singletonMap(outputTag, KvCoder.of(STRING_CODER, IterableCoder.of(STRING_CODER))),
566568
outputTag,
567569
WindowingStrategy.of(window).withTrigger(trigger)
568570
.withMode(ACCUMULATING_FIRED_PANES)

0 commit comments

Comments
 (0)