Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.beam.sdk.util.IllegalMutationException;
import org.apache.beam.sdk.util.MutationDetector;
import org.apache.beam.sdk.util.MutationDetectors;
import org.apache.beam.sdk.util.UserCodeException;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PCollection;

Expand Down Expand Up @@ -111,17 +110,16 @@ public CommittedBundle<T> commit(Instant synchronizedProcessingTime) {
try {
detector.verifyUnmodified();
} catch (IllegalMutationException exn) {
throw UserCodeException.wrap(
new IllegalMutationException(
String.format(
"PTransform %s mutated value %s after it was output (new value was %s)."
+ " Values must not be mutated in any way after being output.",
underlying.getPCollection().getProducingTransformInternal().getFullName(),
exn.getSavedValue(),
exn.getNewValue()),
exn.getSavedValue(),
exn.getNewValue(),
exn));
throw new IllegalMutationException(
String.format(
"PTransform %s mutated value %s after it was output (new value was %s)."
+ " Values must not be mutated in any way after being output.",
underlying.getPCollection().getProducingTransformInternal().getFullName(),
exn.getSavedValue(),
exn.getNewValue()),
exn.getSavedValue(),
exn.getNewValue(),
exn);
}
}
return underlying.commit(synchronizedProcessingTime);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.beam.runners.direct;

import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.isA;
import static org.junit.Assert.assertThat;

import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
Expand All @@ -32,7 +31,6 @@
import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.util.IllegalMutationException;
import org.apache.beam.sdk.util.UserCodeException;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PCollection;

Expand Down Expand Up @@ -168,8 +166,7 @@ public void mutationAfterAddRootBundleThrows() {
root.add(WindowedValue.valueInGlobalWindow(array));

array[1] = 2;
thrown.expect(UserCodeException.class);
thrown.expectCause(isA(IllegalMutationException.class));
thrown.expect(IllegalMutationException.class);
thrown.expectMessage("Values must not be mutated in any way after being output");
CommittedBundle<byte[]> committed = root.commit(Instant.now());
}
Expand All @@ -191,8 +188,7 @@ public void mutationAfterAddKeyedBundleThrows() {
keyed.add(windowedArray);

array[0] = Byte.MAX_VALUE;
thrown.expect(UserCodeException.class);
thrown.expectCause(isA(IllegalMutationException.class));
thrown.expect(IllegalMutationException.class);
thrown.expectMessage("Values must not be mutated in any way after being output");
CommittedBundle<byte[]> committed = keyed.commit(Instant.now());
}
Expand All @@ -212,8 +208,7 @@ public void mutationAfterAddCreateBundleThrows() {
intermediate.add(windowedArray);

array[2] = -3;
thrown.expect(UserCodeException.class);
thrown.expectCause(isA(IllegalMutationException.class));
thrown.expect(IllegalMutationException.class);
thrown.expectMessage("Values must not be mutated in any way after being output");
CommittedBundle<byte[]> committed = intermediate.commit(Instant.now());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.beam.runners.direct.InProcessPipelineRunner.InProcessPipelineResult;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
Expand All @@ -37,13 +38,15 @@
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.util.IllegalMutationException;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
import org.apache.beam.sdk.values.TypeDescriptor;

import com.google.common.collect.ImmutableMap;


import com.fasterxml.jackson.annotation.JsonValue;

import org.junit.Rule;
Expand All @@ -54,6 +57,8 @@
import org.junit.runners.JUnit4;

import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
import java.util.Map;

/**
Expand All @@ -63,6 +68,14 @@
public class InProcessPipelineRunnerTest implements Serializable {
@Rule public transient ExpectedException thrown = ExpectedException.none();

private Pipeline getPipeline() {
PipelineOptions opts = PipelineOptionsFactory.create();
opts.setRunner(InProcessPipelineRunner.class);

Pipeline p = Pipeline.create(opts);
return p;
}

@Test
public void wordCountShouldSucceed() throws Throwable {
Pipeline p = getPipeline();
Expand Down Expand Up @@ -192,11 +205,126 @@ public interface ObjectPipelineOptions extends PipelineOptions {
}


private Pipeline getPipeline() {
PipelineOptions opts = PipelineOptionsFactory.create();
opts.setRunner(InProcessPipelineRunner.class);
/**
* Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
* {@link InProcessPipelineRunner}.
*/
@Test
public void testMutatingOutputThenOutputDoFnError() throws Exception {
Pipeline pipeline = getPipeline();

Pipeline p = Pipeline.create(opts);
return p;
pipeline
.apply(Create.of(42))
.apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
@Override public void processElement(ProcessContext c) {
List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
c.output(outputList);
outputList.set(0, 37);
c.output(outputList);
}
}));

thrown.expect(IllegalMutationException.class);
thrown.expectMessage("output");
thrown.expectMessage("must not be mutated");
pipeline.run();
}

/**
* Tests that a {@link DoFn} that mutates an output with a good equals() fails in the
* {@link InProcessPipelineRunner}.
*/
@Test
public void testMutatingOutputThenTerminateDoFnError() throws Exception {
Pipeline pipeline = getPipeline();

pipeline
.apply(Create.of(42))
.apply(ParDo.of(new DoFn<Integer, List<Integer>>() {
@Override public void processElement(ProcessContext c) {
List<Integer> outputList = Arrays.asList(1, 2, 3, 4);
c.output(outputList);
outputList.set(0, 37);
}
}));

thrown.expect(IllegalMutationException.class);
thrown.expectMessage("output");
thrown.expectMessage("must not be mutated");
pipeline.run();
}

/**
* Tests that a {@link DoFn} that mutates an output with a bad equals() still fails
* in the {@link InProcessPipelineRunner}.
*/
@Test
public void testMutatingOutputCoderDoFnError() throws Exception {
Pipeline pipeline = getPipeline();

pipeline
.apply(Create.of(42))
.apply(ParDo.of(new DoFn<Integer, byte[]>() {
@Override public void processElement(ProcessContext c) {
byte[] outputArray = new byte[]{0x1, 0x2, 0x3};
c.output(outputArray);
outputArray[0] = 0xa;
c.output(outputArray);
}
}));

thrown.expect(IllegalMutationException.class);
thrown.expectMessage("output");
thrown.expectMessage("must not be mutated");
pipeline.run();
}

/**
* Tests that a {@link DoFn} that mutates its input with a good equals() fails in the
* {@link InProcessPipelineRunner}.
*/
@Test
public void testMutatingInputDoFnError() throws Exception {
Pipeline pipeline = getPipeline();

pipeline
.apply(Create.of(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))
.withCoder(ListCoder.of(VarIntCoder.of())))
.apply(ParDo.of(new DoFn<List<Integer>, Integer>() {
@Override public void processElement(ProcessContext c) {
List<Integer> inputList = c.element();
inputList.set(0, 37);
c.output(12);
}
}));

thrown.expect(IllegalMutationException.class);
thrown.expectMessage("Input");
thrown.expectMessage("must not be mutated");
pipeline.run();
}

/**
* Tests that a {@link DoFn} that mutates an input with a bad equals() still fails
* in the {@link InProcessPipelineRunner}.
*/
@Test
public void testMutatingInputCoderDoFnError() throws Exception {
Pipeline pipeline = getPipeline();

pipeline
.apply(Create.of(new byte[]{0x1, 0x2, 0x3}, new byte[]{0x4, 0x5, 0x6}))
.apply(ParDo.of(new DoFn<byte[], Integer>() {
@Override public void processElement(ProcessContext c) {
byte[] inputArray = c.element();
inputArray[0] = 0xa;
c.output(13);
}
}));

thrown.expect(IllegalMutationException.class);
thrown.expectMessage("Input");
thrown.expectMessage("must not be mutated");
pipeline.run();
}
}
Loading