Add UnsupportedOverrideFactory to core-construction This factory throws with a specified message for all of its methods.
Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/f29e4c36 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/f29e4c36 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/f29e4c36 Branch: refs/heads/master Commit: f29e4c36454ccf1f35da7e314008ba898b6983c6 Parents: e75fbe7 Author: Thomas Groh <[email protected]> Authored: Thu Feb 16 09:47:50 2017 -0800 Committer: Thomas Groh <[email protected]> Committed: Fri Feb 17 14:21:10 2017 -0800 ---------------------------------------------------------------------- .../UnsupportedOverrideFactory.java | 73 ++++++++++++++++++++ .../UnsupportedOverrideFactoryTest.java | 65 +++++++++++++++++ 2 files changed, 138 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/f29e4c36/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java new file mode 100644 index 0000000..2072574 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.core.construction; + +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; + +/** + * A {@link PTransformOverrideFactory} that throws an exception when a call to + * {@link #getReplacementTransform(PTransform)} is made. This is for {@link PTransform PTransforms} + * which are not supported by a runner. + */ +public final class UnsupportedOverrideFactory< + InputT extends PInput, + OutputT extends POutput, + TransformT extends PTransform<InputT, OutputT>> + implements PTransformOverrideFactory<InputT, OutputT, TransformT> { + + private final String message; + + @SuppressWarnings("rawtypes") + public static < + InputT extends PInput, + OutputT extends POutput, + TransformT extends PTransform<InputT, OutputT>> + UnsupportedOverrideFactory<InputT, OutputT, TransformT> withMessage(String message) { + return new UnsupportedOverrideFactory<>(message); + } + + private UnsupportedOverrideFactory(String message) { + this.message = message; + } + + @Override + public PTransform<InputT, OutputT> getReplacementTransform(TransformT transform) { + throw new UnsupportedOperationException(message); + } + + @Override + public InputT getInput( + List<TaggedPValue> inputs, Pipeline p) { + throw new UnsupportedOperationException(message); + } + + @Override + public Map<PValue, ReplacementOutput> mapOutputs( + List<TaggedPValue> outputs, OutputT newOutput) { + throw new UnsupportedOperationException(message); + } +} http://git-wip-us.apache.org/repos/asf/beam/blob/f29e4c36/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java ---------------------------------------------------------------------- diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java new file mode 100644 index 0000000..f33d0f9 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.core.construction; + +import java.util.Collections; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TaggedPValue; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link UnsupportedOverrideFactory}. + */ +@RunWith(JUnit4.class) +public class UnsupportedOverrideFactoryTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + private final String message = "my_error_message"; + private TestPipeline pipeline = TestPipeline.create(); + private UnsupportedOverrideFactory factory = + UnsupportedOverrideFactory.withMessage(message); + + @Test + public void getReplacementTransformThrows() { + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage(message); + factory.getReplacementTransform(Create.empty(VoidCoder.of())); + } + + @Test + public void getInputThrows() { + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage(message); + factory.getInput(Collections.<TaggedPValue>emptyList(), pipeline); + } + + @Test + public void mapOutputThrows() { + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage(message); + factory.mapOutputs(Collections.<TaggedPValue>emptyList(), PDone.in(pipeline)); + } +}
