forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
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
4 changed files
with
205 additions
and
0 deletions.
There are no files selected for viewing
72 changes: 72 additions & 0 deletions
72
...ruction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.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,72 @@ | ||
/* | ||
* 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.HashSet; | ||
import java.util.Set; | ||
import org.apache.beam.sdk.Pipeline; | ||
import org.apache.beam.sdk.Pipeline.PipelineVisitor; | ||
import org.apache.beam.sdk.io.Read; | ||
import org.apache.beam.sdk.runners.TransformHierarchy.Node; | ||
import org.apache.beam.sdk.transforms.DoFn; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.transforms.ParDo; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.PValue; | ||
|
||
/** | ||
* Utilities for ensuring that all {@link Read} {@link PTransform PTransforms} are consumed by some | ||
* {@link PTransform}. | ||
*/ | ||
public class UnconsumedReads { | ||
public static void ensureAllReadsConsumed(Pipeline pipeline) { | ||
final Set<PCollection<?>> unconsumed = new HashSet<>(); | ||
pipeline.traverseTopologically( | ||
new PipelineVisitor.Defaults() { | ||
@Override | ||
public void visitPrimitiveTransform(Node node) { | ||
unconsumed.removeAll(node.getInputs().values()); | ||
} | ||
|
||
@Override | ||
public void visitValue(PValue value, Node producer) { | ||
if (producer.getTransform() instanceof Read.Bounded | ||
|| producer.getTransform() instanceof Read.Unbounded) { | ||
unconsumed.add((PCollection<?>) value); | ||
} | ||
} | ||
}); | ||
int i = 0; | ||
for (PCollection<?> unconsumedPCollection : unconsumed) { | ||
consume(unconsumedPCollection, i); | ||
i++; | ||
} | ||
} | ||
|
||
private static <T> void consume(PCollection<T> unconsumedPCollection, int uniq) { | ||
// Multiple applications should never break due to stable unique names. | ||
String uniqueName = "DropInputs" + (uniq == 0 ? "" : uniq); | ||
unconsumedPCollection.apply(uniqueName, ParDo.of(new NoOpDoFn<T>())); | ||
} | ||
|
||
private static class NoOpDoFn<T> extends DoFn<T, T> { | ||
@ProcessElement | ||
public void doNothing(ProcessContext context) {} | ||
} | ||
} |
105 changes: 105 additions & 0 deletions
105
...ion-java/src/test/java/org/apache/beam/runners/core/construction/UnconsumedReadsTest.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,105 @@ | ||
/* | ||
* 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.HashSet; | ||
import java.util.Set; | ||
import org.apache.beam.sdk.Pipeline.PipelineVisitor; | ||
import org.apache.beam.sdk.io.CountingSource; | ||
import org.apache.beam.sdk.io.Read; | ||
import org.apache.beam.sdk.io.Read.Bounded; | ||
import org.apache.beam.sdk.io.Read.Unbounded; | ||
import org.apache.beam.sdk.runners.TransformHierarchy.Node; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
import org.apache.beam.sdk.transforms.Flatten; | ||
import org.apache.beam.sdk.transforms.PTransform; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.PCollectionList; | ||
import org.apache.beam.sdk.values.PValue; | ||
import org.hamcrest.Matchers; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.junit.runners.JUnit4; | ||
|
||
/** | ||
* Tests for {@link UnconsumedReads}. | ||
*/ | ||
@RunWith(JUnit4.class) | ||
public class UnconsumedReadsTest { | ||
@Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); | ||
|
||
@Test | ||
public void matcherProducesUnconsumedValueBoundedRead() { | ||
Bounded<Long> transform = Read.from(CountingSource.upTo(20L)); | ||
PCollection<Long> output = pipeline.apply(transform); | ||
UnconsumedReads.ensureAllReadsConsumed(pipeline); | ||
validateConsumed(); | ||
} | ||
|
||
@Test | ||
public void matcherProducesUnconsumedValueUnboundedRead() { | ||
Unbounded<Long> transform = Read.from(CountingSource.unbounded()); | ||
PCollection<Long> output = pipeline.apply(transform); | ||
UnconsumedReads.ensureAllReadsConsumed(pipeline); | ||
validateConsumed(); | ||
} | ||
|
||
@Test | ||
public void doesNotConsumeAlreadyConsumedRead() { | ||
Unbounded<Long> transform = Read.from(CountingSource.unbounded()); | ||
final PCollection<Long> output = pipeline.apply(transform); | ||
final Flatten.PCollections<Long> consumer = Flatten.<Long>pCollections(); | ||
PCollectionList.of(output).apply(consumer); | ||
UnconsumedReads.ensureAllReadsConsumed(pipeline); | ||
pipeline.traverseTopologically( | ||
new PipelineVisitor.Defaults() { | ||
@Override | ||
public void visitPrimitiveTransform(Node node) { | ||
// The output should only be consumed by a single consumer | ||
if (node.getInputs().values().contains(output)) { | ||
assertThat(node.getTransform(), Matchers.<PTransform<?, ?>>is(consumer)); | ||
} | ||
} | ||
}); | ||
} | ||
|
||
private void validateConsumed() { | ||
final Set<PValue> consumedOutputs = new HashSet<PValue>(); | ||
final Set<PValue> allReadOutputs = new HashSet<PValue>(); | ||
pipeline.traverseTopologically( | ||
new PipelineVisitor.Defaults() { | ||
@Override | ||
public void visitPrimitiveTransform(Node node) { | ||
consumedOutputs.addAll(node.getInputs().values()); | ||
} | ||
|
||
@Override | ||
public void visitValue(PValue value, Node producer) { | ||
if (producer.getTransform() instanceof Read.Bounded | ||
|| producer.getTransform() instanceof Read.Unbounded) { | ||
allReadOutputs.add(value); | ||
} | ||
} | ||
}); | ||
assertThat(consumedOutputs, Matchers.hasItems(allReadOutputs.toArray(new PValue[0]))); | ||
} | ||
} |
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