Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
20 changed files
with
594 additions
and
52 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
50 changes: 50 additions & 0 deletions
50
...ruction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* 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 static com.google.common.base.Preconditions.checkArgument; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import java.util.Collection; | ||
import java.util.Map; | ||
import org.apache.beam.sdk.runners.AppliedPTransform; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.values.PValue; | ||
import org.apache.beam.sdk.values.TupleTag; | ||
|
||
/** Utilities for extracting subsets of inputs from an {@link AppliedPTransform}. */ | ||
public class TransformInputs { | ||
/** | ||
* Gets all inputs of the {@link AppliedPTransform} that are not returned by {@link | ||
* PTransform#getAdditionalInputs()}. | ||
*/ | ||
public static Collection<PValue> nonAdditionalInputs(AppliedPTransform<?, ?, ?> application) { | ||
ImmutableList.Builder<PValue> mainInputs = ImmutableList.builder(); | ||
PTransform<?, ?> transform = application.getTransform(); | ||
for (Map.Entry<TupleTag<?>, PValue> input : application.getInputs().entrySet()) { | ||
if (!transform.getAdditionalInputs().containsKey(input.getKey())) { | ||
mainInputs.add(input.getValue()); | ||
} | ||
} | ||
checkArgument( | ||
!mainInputs.build().isEmpty() || application.getInputs().isEmpty(), | ||
"Expected at least one main input if any inputs exist"); | ||
return mainInputs.build(); | ||
} | ||
} |
166 changes: 166 additions & 0 deletions
166
...ion-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,166 @@ | ||
/* | ||
* 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 static org.junit.Assert.assertThat; | ||
|
||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import org.apache.beam.sdk.coders.VoidCoder; | ||
import org.apache.beam.sdk.io.GenerateSequence; | ||
import org.apache.beam.sdk.runners.AppliedPTransform; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
import org.apache.beam.sdk.transforms.Create; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.PDone; | ||
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.TupleTag; | ||
import org.hamcrest.Matchers; | ||
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 TransformInputs}. */ | ||
@RunWith(JUnit4.class) | ||
public class TransformInputsTest { | ||
@Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); | ||
@Rule public ExpectedException thrown = ExpectedException.none(); | ||
|
||
@Test | ||
public void nonAdditionalInputsWithNoInputSucceeds() { | ||
AppliedPTransform<PInput, POutput, TestTransform> transform = | ||
AppliedPTransform.of( | ||
"input-free", | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
new TestTransform(), | ||
pipeline); | ||
|
||
assertThat(TransformInputs.nonAdditionalInputs(transform), Matchers.<PValue>empty()); | ||
} | ||
|
||
@Test | ||
public void nonAdditionalInputsWithOneMainInputSucceeds() { | ||
PCollection<Long> input = pipeline.apply(GenerateSequence.from(1L)); | ||
AppliedPTransform<PInput, POutput, TestTransform> transform = | ||
AppliedPTransform.of( | ||
"input-single", | ||
Collections.<TupleTag<?>, PValue>singletonMap(new TupleTag<Long>() {}, input), | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
new TestTransform(), | ||
pipeline); | ||
|
||
assertThat( | ||
TransformInputs.nonAdditionalInputs(transform), Matchers.<PValue>containsInAnyOrder(input)); | ||
} | ||
|
||
@Test | ||
public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() { | ||
Map<TupleTag<?>, PValue> allInputs = new HashMap<>(); | ||
PCollection<Integer> mainInts = pipeline.apply("MainInput", Create.of(12, 3)); | ||
allInputs.put(new TupleTag<Integer>() {}, mainInts); | ||
PCollection<Void> voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); | ||
allInputs.put(new TupleTag<Void>() {}, voids); | ||
AppliedPTransform<PInput, POutput, TestTransform> transform = | ||
AppliedPTransform.of( | ||
"additional-free", | ||
allInputs, | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
new TestTransform(), | ||
pipeline); | ||
|
||
assertThat( | ||
TransformInputs.nonAdditionalInputs(transform), | ||
Matchers.<PValue>containsInAnyOrder(voids, mainInts)); | ||
} | ||
|
||
@Test | ||
public void nonAdditionalInputsWithAdditionalInputsSucceeds() { | ||
Map<TupleTag<?>, PValue> additionalInputs = new HashMap<>(); | ||
additionalInputs.put(new TupleTag<String>() {}, pipeline.apply(Create.of("1, 2", "3"))); | ||
additionalInputs.put(new TupleTag<Long>() {}, pipeline.apply(GenerateSequence.from(3L))); | ||
|
||
Map<TupleTag<?>, PValue> allInputs = new HashMap<>(); | ||
PCollection<Integer> mainInts = pipeline.apply("MainInput", Create.of(12, 3)); | ||
allInputs.put(new TupleTag<Integer>() {}, mainInts); | ||
PCollection<Void> voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); | ||
allInputs.put( | ||
new TupleTag<Void>() {}, voids); | ||
allInputs.putAll(additionalInputs); | ||
|
||
AppliedPTransform<PInput, POutput, TestTransform> transform = | ||
AppliedPTransform.of( | ||
"additional", | ||
allInputs, | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
new TestTransform(additionalInputs), | ||
pipeline); | ||
|
||
assertThat( | ||
TransformInputs.nonAdditionalInputs(transform), | ||
Matchers.<PValue>containsInAnyOrder(mainInts, voids)); | ||
} | ||
|
||
@Test | ||
public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() { | ||
Map<TupleTag<?>, PValue> additionalInputs = new HashMap<>(); | ||
additionalInputs.put(new TupleTag<String>() {}, pipeline.apply(Create.of("1, 2", "3"))); | ||
additionalInputs.put(new TupleTag<Long>() {}, pipeline.apply(GenerateSequence.from(3L))); | ||
|
||
AppliedPTransform<PInput, POutput, TestTransform> transform = | ||
AppliedPTransform.of( | ||
"additional-only", | ||
additionalInputs, | ||
Collections.<TupleTag<?>, PValue>emptyMap(), | ||
new TestTransform(additionalInputs), | ||
pipeline); | ||
|
||
thrown.expect(IllegalArgumentException.class); | ||
thrown.expectMessage("at least one"); | ||
TransformInputs.nonAdditionalInputs(transform); | ||
} | ||
|
||
private static class TestTransform extends PTransform<PInput, POutput> { | ||
private final Map<TupleTag<?>, PValue> additionalInputs; | ||
|
||
private TestTransform() { | ||
this(Collections.<TupleTag<?>, PValue>emptyMap()); | ||
} | ||
|
||
private TestTransform(Map<TupleTag<?>, PValue> additionalInputs) { | ||
this.additionalInputs = additionalInputs; | ||
} | ||
|
||
@Override | ||
public POutput expand(PInput input) { | ||
return PDone.in(input.getPipeline()); | ||
} | ||
|
||
@Override | ||
public Map<TupleTag<?>, PValue> getAdditionalInputs() { | ||
return additionalInputs; | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.